You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/04/08 13:24:50 UTC

[GitHub] [flink] godfreyhe opened a new pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator

godfreyhe opened a new pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671
 
 
   
   ## What is the purpose of the change
   
   *As [FLINK-16533](https://issues.apache.org/jira/browse/FLINK-16533) discussed, We move the most part logic of ExecutionEnvironment#createProgramPlan method to PlanGenerator, which can be used by ExecutionEnvironment and flink-table-planner.*
   
   
   ## Brief change log
   
     - *extract plan generation logic from ExecutionEnvironment#createProgramPlan to PlanGenerator*
   
   
   ## Verifying this change
   
   *(Please pick either of the following options)*
   
   This change is already covered by existing tests, such as *CollectionExecutionAccumulatorsTest, ExecutorDiscoveryAndJobClientTest*. And Added PlanGeneratorTest can also verify the logic.
    
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (yes / **no**)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (yes / **no**)
     - The serializers: (yes / **no** / don't know)
     - The runtime per-record code paths (performance sensitive): (yes / **no** / don't know)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: (yes / **no** / don't know)
     - The S3 file system connector: (yes / **no** / don't know)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (yes / **no**)
     - If yes, how is the feature documented? (not applicable / docs / JavaDocs / **not documented**)
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kl0u commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
kl0u commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#discussion_r407994603
 
 

 ##########
 File path: flink-java/src/main/java/org/apache/flink/api/java/utils/PlanGenerator.java
 ##########
 @@ -0,0 +1,126 @@
+/*
+ * 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.java.utils;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.Plan;
+import org.apache.flink.api.common.operators.Operator;
+import org.apache.flink.api.common.operators.OperatorInformation;
+import org.apache.flink.api.java.operators.DataSink;
+import org.apache.flink.api.java.operators.OperatorTranslation;
+import org.apache.flink.api.java.typeutils.runtime.kryo.Serializers;
+import org.apache.flink.util.Visitor;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * A generator that generates a {@link Plan} from a graph of {@link Operator}s.
+ */
+public class PlanGenerator {
+	private static final Logger LOG = LoggerFactory.getLogger(PlanGenerator.class);
+
+	private final List<DataSink<?>> sinks;
+	private final ExecutionConfig config;
+	private final String jobName;
+
+	public PlanGenerator(
+			List<DataSink<?>> sinks,
+			ExecutionConfig config,
+			String jobName) {
+		this.sinks = sinks;
+		this.config = config;
+		this.jobName = jobName;
+	}
+
+	public Plan generate() {
+		OperatorTranslation translator = new OperatorTranslation();
+		Plan plan = translator.translateToPlan(sinks, jobName);
+
+		if (config.getParallelism() > 0) {
+			plan.setDefaultParallelism(config.getParallelism());
+		}
+		plan.setExecutionConfig(config);
+
+		// Check plan for GenericTypeInfo's and register the types at the serializers.
+		if (!config.isAutoTypeRegistrationDisabled()) {
+			plan.accept(new Visitor<Operator<?>>() {
+
+				private final Set<Class<?>> registeredTypes = new HashSet<>();
+				private final Set<org.apache.flink.api.common.operators.Operator<?>> visitedOperators = new HashSet<>();
+
+				@Override
+				public boolean preVisit(org.apache.flink.api.common.operators.Operator<?> visitable) {
+					if (!visitedOperators.add(visitable)) {
+						return false;
+					}
+					OperatorInformation<?> opInfo = visitable.getOperatorInfo();
+					Serializers.recursivelyRegisterType(opInfo.getOutputType(), config, registeredTypes);
+					return true;
+				}
+
+				@Override
+				public void postVisit(org.apache.flink.api.common.operators.Operator<?> visitable) {
+				}
+			});
+		}
+
+		// All types are registered now. Print information.
+		int registeredTypes = config.getRegisteredKryoTypes().size() +
+				config.getRegisteredPojoTypes().size() +
+				config.getRegisteredTypesWithKryoSerializerClasses().size() +
+				config.getRegisteredTypesWithKryoSerializers().size();
+		int defaultKryoSerializers = config.getDefaultKryoSerializers().size() +
+				config.getDefaultKryoSerializerClasses().size();
+		LOG.info("The job has {} registered types and {} default Kryo serializers", registeredTypes,
+				defaultKryoSerializers);
+
+		if (config.isForceKryoEnabled() && config.isForceAvroEnabled()) {
+			LOG.warn("In the ExecutionConfig, both Avro and Kryo are enforced. Using Kryo serializer");
+		}
+		if (config.isForceKryoEnabled()) {
+			LOG.info("Using KryoSerializer for serializing POJOs");
+		}
+		if (config.isForceAvroEnabled()) {
+			LOG.info("Using AvroSerializer for serializing POJOs");
+		}
+
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("Registered Kryo types: {}", config.getRegisteredKryoTypes().toString());
+			LOG.debug("Registered Kryo with Serializers types: {}",
+					config.getRegisteredTypesWithKryoSerializers().entrySet().toString());
+			LOG.debug("Registered Kryo with Serializer Classes types: {}",
+					config.getRegisteredTypesWithKryoSerializerClasses().entrySet().toString());
+			LOG.debug("Registered Kryo default Serializers: {}",
+					config.getDefaultKryoSerializers().entrySet().toString());
+			LOG.debug("Registered Kryo default Serializers Classes {}",
+					config.getDefaultKryoSerializerClasses().entrySet().toString());
+			LOG.debug("Registered POJO types: {}", config.getRegisteredPojoTypes().toString());
+
+			// print information about static code analysis
+			LOG.debug("Static code analysis mode: {}", config.getCodeAnalysisMode());
 
 Review comment:
   This could go in a separate method like: `logDebuggingTypeDetails()`.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159593526",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/159593526) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kl0u closed pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
kl0u closed pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671
 
 
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159593526",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159725289",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc01102e1595476cea82051bb992acb9427da257",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7357",
       "triggerID" : "bc01102e1595476cea82051bb992acb9427da257",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "612549149",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "612549149",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "612735854",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "612735854",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "613282091",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "613282091",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b7adab126357d7fb74426dcb9b5fb0b765d8eead Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/159866074) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343) 
   * 5110feb83332d8d0583a2be9534ac1565ef0be7b UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159593526",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159725289",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc01102e1595476cea82051bb992acb9427da257",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7357",
       "triggerID" : "bc01102e1595476cea82051bb992acb9427da257",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "612549149",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "612549149",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "612735854",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "612735854",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * b7adab126357d7fb74426dcb9b5fb0b765d8eead Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/159866074) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159593526",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159725289",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc01102e1595476cea82051bb992acb9427da257",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7357",
       "triggerID" : "bc01102e1595476cea82051bb992acb9427da257",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "612549149",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "612549149",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "612735854",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "612735854",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "613282091",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "613282091",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/160182477",
       "triggerID" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7449",
       "triggerID" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b90f363254f04c75de526ddb0e3a2bed9686373",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2b90f363254f04c75de526ddb0e3a2bed9686373",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5110feb83332d8d0583a2be9534ac1565ef0be7b Travis: [CANCELED](https://travis-ci.com/github/flink-ci/flink/builds/160182477) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7449) 
   * 2b90f363254f04c75de526ddb0e3a2bed9686373 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] godfreyhe commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-613346038
 
 
   > @godfreyhe I just saw you last commit, I think it was a nice change to pass the `cacheFile` in the constructor of the `PlanGenerator` and register the files there. This delegates the whole process of the "plan creation" to the new class.
   > 
   > My comment was just to not expose it through the new public method :)
   
   oh, I misunderstand you concern. `PlanGenerator` also supports  cache registration. but for `TableEnvironment`, the argument of cacheFiles are alway empty, even if `ExecutionEnvironment` has cache files.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] godfreyhe commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-613282091
 
 
   @flinkbot run travis

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] godfreyhe commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#discussion_r407983968
 
 

 ##########
 File path: flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
 ##########
 @@ -1035,6 +1030,14 @@ protected void registerCachedFilesWithPlan(Plan p) throws IOException {
 		}
 	}
 
+	/**
 
 Review comment:
   Make sense to me, I will revert it

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] godfreyhe commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-613305143
 
 
   my private travis is passed https://travis-ci.org/github/godfreyhe/flink/builds/673799271

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b7e5ae168bcf572e6321257246a778951e5bb6a0 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/159317260) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230) 
   * 0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159593526",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159725289",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc01102e1595476cea82051bb992acb9427da257",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7357",
       "triggerID" : "bc01102e1595476cea82051bb992acb9427da257",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "612549149",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "612549149",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "612735854",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "612735854",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "613282091",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "613282091",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/160182477",
       "triggerID" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7449",
       "triggerID" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b90f363254f04c75de526ddb0e3a2bed9686373",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2b90f363254f04c75de526ddb0e3a2bed9686373",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e2bccdeb54eb8066d91325213a8d30a1d292e96",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/160684351",
       "triggerID" : "7e2bccdeb54eb8066d91325213a8d30a1d292e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e2bccdeb54eb8066d91325213a8d30a1d292e96",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7635",
       "triggerID" : "7e2bccdeb54eb8066d91325213a8d30a1d292e96",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2b90f363254f04c75de526ddb0e3a2bed9686373 UNKNOWN
   * 7e2bccdeb54eb8066d91325213a8d30a1d292e96 Travis: [SUCCESS](https://travis-ci.com/github/flink-ci/flink/builds/160684351) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7635) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] godfreyhe commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-612735854
 
 
   @flinkbot run travis

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159593526",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b7e5ae168bcf572e6321257246a778951e5bb6a0 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/159317260) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230) 
   * 0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/159593526) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159593526",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159725289",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc01102e1595476cea82051bb992acb9427da257",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7357",
       "triggerID" : "bc01102e1595476cea82051bb992acb9427da257",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "612549149",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "612549149",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "612735854",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "612735854",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "613282091",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "613282091",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/160182477",
       "triggerID" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7449",
       "triggerID" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b90f363254f04c75de526ddb0e3a2bed9686373",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2b90f363254f04c75de526ddb0e3a2bed9686373",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b7adab126357d7fb74426dcb9b5fb0b765d8eead Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/159866074) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343) 
   * 5110feb83332d8d0583a2be9534ac1565ef0be7b Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/160182477) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7449) 
   * 2b90f363254f04c75de526ddb0e3a2bed9686373 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159593526",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159725289",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f82a2583fb82c50376de97b616de11db678d88e6 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/159725289) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] godfreyhe commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-612735864
 
 
   @flinkbot run azure

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159593526",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159725289",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc01102e1595476cea82051bb992acb9427da257",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7357",
       "triggerID" : "bc01102e1595476cea82051bb992acb9427da257",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "612549149",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "612549149",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "612735854",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "612735854",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "613282091",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * b7adab126357d7fb74426dcb9b5fb0b765d8eead Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/159866074) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] godfreyhe commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#discussion_r408539404
 
 

 ##########
 File path: flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/TestUtils.java
 ##########
 @@ -91,8 +91,6 @@ private TestUtils() {}
 	 * @param expectedParallelism expected operator parallelism
 	 */
 	public static void verifyParallelism(ExecutionEnvironment env, int expectedParallelism) {
-		env.setParallelism(2 * expectedParallelism);
 
 Review comment:
   I think so. `CollectionEnvironment` is a special case that requires the parallelism is always 1.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kl0u commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
kl0u commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#discussion_r407993771
 
 

 ##########
 File path: flink-java/src/main/java/org/apache/flink/api/java/utils/PlanGenerator.java
 ##########
 @@ -0,0 +1,126 @@
+/*
+ * 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.java.utils;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.Plan;
+import org.apache.flink.api.common.operators.Operator;
+import org.apache.flink.api.common.operators.OperatorInformation;
+import org.apache.flink.api.java.operators.DataSink;
+import org.apache.flink.api.java.operators.OperatorTranslation;
+import org.apache.flink.api.java.typeutils.runtime.kryo.Serializers;
+import org.apache.flink.util.Visitor;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * A generator that generates a {@link Plan} from a graph of {@link Operator}s.
+ */
+public class PlanGenerator {
+	private static final Logger LOG = LoggerFactory.getLogger(PlanGenerator.class);
+
+	private final List<DataSink<?>> sinks;
+	private final ExecutionConfig config;
+	private final String jobName;
+
+	public PlanGenerator(
+			List<DataSink<?>> sinks,
+			ExecutionConfig config,
+			String jobName) {
+		this.sinks = sinks;
+		this.config = config;
+		this.jobName = jobName;
+	}
+
+	public Plan generate() {
+		OperatorTranslation translator = new OperatorTranslation();
+		Plan plan = translator.translateToPlan(sinks, jobName);
+
+		if (config.getParallelism() > 0) {
+			plan.setDefaultParallelism(config.getParallelism());
+		}
+		plan.setExecutionConfig(config);
+
+		// Check plan for GenericTypeInfo's and register the types at the serializers.
+		if (!config.isAutoTypeRegistrationDisabled()) {
+			plan.accept(new Visitor<Operator<?>>() {
+
+				private final Set<Class<?>> registeredTypes = new HashSet<>();
+				private final Set<org.apache.flink.api.common.operators.Operator<?>> visitedOperators = new HashSet<>();
+
+				@Override
+				public boolean preVisit(org.apache.flink.api.common.operators.Operator<?> visitable) {
+					if (!visitedOperators.add(visitable)) {
+						return false;
+					}
+					OperatorInformation<?> opInfo = visitable.getOperatorInfo();
+					Serializers.recursivelyRegisterType(opInfo.getOutputType(), config, registeredTypes);
+					return true;
+				}
+
+				@Override
+				public void postVisit(org.apache.flink.api.common.operators.Operator<?> visitable) {
+				}
+			});
+		}
+
+		// All types are registered now. Print information.
+		int registeredTypes = config.getRegisteredKryoTypes().size() +
+				config.getRegisteredPojoTypes().size() +
+				config.getRegisteredTypesWithKryoSerializerClasses().size() +
+				config.getRegisteredTypesWithKryoSerializers().size();
 
 Review comment:
   This could go in a separate method like: `getNumberOfRegisteredTypes()`

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] godfreyhe commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-612549149
 
 
   @flinkbot run travis

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b7e5ae168bcf572e6321257246a778951e5bb6a0 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/159317260) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] godfreyhe commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#discussion_r408843301
 
 

 ##########
 File path: flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/TestUtils.java
 ##########
 @@ -91,8 +91,6 @@ private TestUtils() {}
 	 * @param expectedParallelism expected operator parallelism
 	 */
 	public static void verifyParallelism(ExecutionEnvironment env, int expectedParallelism) {
-		env.setParallelism(2 * expectedParallelism);
 
 Review comment:
   hi @kl0u does this make sense to you ?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kl0u commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
kl0u commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-613278490
 
 
   BTW both builds (Azure and Travis) seem to be failing. Have you checked them?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159593526",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159725289",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b7adab126357d7fb74426dcb9b5fb0b765d8eead Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/159866074) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] godfreyhe commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#discussion_r407302559
 
 

 ##########
 File path: flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
 ##########
 @@ -1035,6 +1030,14 @@ protected void registerCachedFilesWithPlan(Plan p) throws IOException {
 		}
 	}
 
+	/**
 
 Review comment:
   hi @aljoscha, do you have any other concern about this ?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kl0u commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
kl0u commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#discussion_r407994050
 
 

 ##########
 File path: flink-java/src/main/java/org/apache/flink/api/java/utils/PlanGenerator.java
 ##########
 @@ -0,0 +1,126 @@
+/*
+ * 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.java.utils;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.Plan;
+import org.apache.flink.api.common.operators.Operator;
+import org.apache.flink.api.common.operators.OperatorInformation;
+import org.apache.flink.api.java.operators.DataSink;
+import org.apache.flink.api.java.operators.OperatorTranslation;
+import org.apache.flink.api.java.typeutils.runtime.kryo.Serializers;
+import org.apache.flink.util.Visitor;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * A generator that generates a {@link Plan} from a graph of {@link Operator}s.
+ */
+public class PlanGenerator {
+	private static final Logger LOG = LoggerFactory.getLogger(PlanGenerator.class);
+
+	private final List<DataSink<?>> sinks;
+	private final ExecutionConfig config;
+	private final String jobName;
+
+	public PlanGenerator(
+			List<DataSink<?>> sinks,
+			ExecutionConfig config,
+			String jobName) {
+		this.sinks = sinks;
+		this.config = config;
+		this.jobName = jobName;
+	}
+
+	public Plan generate() {
+		OperatorTranslation translator = new OperatorTranslation();
+		Plan plan = translator.translateToPlan(sinks, jobName);
+
+		if (config.getParallelism() > 0) {
+			plan.setDefaultParallelism(config.getParallelism());
+		}
+		plan.setExecutionConfig(config);
+
+		// Check plan for GenericTypeInfo's and register the types at the serializers.
+		if (!config.isAutoTypeRegistrationDisabled()) {
+			plan.accept(new Visitor<Operator<?>>() {
+
+				private final Set<Class<?>> registeredTypes = new HashSet<>();
+				private final Set<org.apache.flink.api.common.operators.Operator<?>> visitedOperators = new HashSet<>();
+
+				@Override
+				public boolean preVisit(org.apache.flink.api.common.operators.Operator<?> visitable) {
+					if (!visitedOperators.add(visitable)) {
+						return false;
+					}
+					OperatorInformation<?> opInfo = visitable.getOperatorInfo();
+					Serializers.recursivelyRegisterType(opInfo.getOutputType(), config, registeredTypes);
+					return true;
+				}
+
+				@Override
+				public void postVisit(org.apache.flink.api.common.operators.Operator<?> visitable) {
+				}
+			});
+		}
+
+		// All types are registered now. Print information.
+		int registeredTypes = config.getRegisteredKryoTypes().size() +
+				config.getRegisteredPojoTypes().size() +
+				config.getRegisteredTypesWithKryoSerializerClasses().size() +
+				config.getRegisteredTypesWithKryoSerializers().size();
+		int defaultKryoSerializers = config.getDefaultKryoSerializers().size() +
+				config.getDefaultKryoSerializerClasses().size();
 
 Review comment:
   This could go in a separate method like: `getNumberOfDefaultKryoSerializers()`

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] godfreyhe commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-613281998
 
 
   that is the penultimate commit (b7adab1) failure,  Azure and Travis don't run my latest commit (bc01102). I re-trigger tests

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] godfreyhe commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#discussion_r408067969
 
 

 ##########
 File path: flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/TestUtils.java
 ##########
 @@ -91,8 +91,6 @@ private TestUtils() {}
 	 * @param expectedParallelism expected operator parallelism
 	 */
 	public static void verifyParallelism(ExecutionEnvironment env, int expectedParallelism) {
-		env.setParallelism(2 * expectedParallelism);
 
 Review comment:
   yes. before this pr, `CollectionEnvironment` override `getParallelism` method, and always return 1.  `ExecutionEnvironment#createProgramPlan` get parallelism through `ExecutionEnvironment#getParallelism`. Now, `PlanGenerator` get parallelism through `config.getParallelism()`.  When `CollectionEnvironment` is created in `ExecutionEnvironment#createCollectionsEnvironment` method, the parallelism in config will be set as 1, while after `env.setParallelism(2 * expectedParallelism);` is called, the parallelism in config was changed to 4. which is unexpected.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] godfreyhe commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-613186963
 
 
   hi @kl0u,  do you have time to help review this pr ?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] godfreyhe commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#discussion_r406145008
 
 

 ##########
 File path: flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
 ##########
 @@ -1035,6 +1030,14 @@ protected void registerCachedFilesWithPlan(Plan p) throws IOException {
 		}
 	}
 
+	/**
 
 Review comment:
   I find `ExecutionConfig` has forwarded everything except `cacheFile`. CMIIW

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kl0u commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
kl0u commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#discussion_r408253636
 
 

 ##########
 File path: flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/TestUtils.java
 ##########
 @@ -91,8 +91,6 @@ private TestUtils() {}
 	 * @param expectedParallelism expected operator parallelism
 	 */
 	public static void verifyParallelism(ExecutionEnvironment env, int expectedParallelism) {
-		env.setParallelism(2 * expectedParallelism);
 
 Review comment:
   I agree that the test design is a bit confusing. But doesn't this mean that this PR changes existing behavior?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159593526",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159725289",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/159593526) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283) 
   * f82a2583fb82c50376de97b616de11db678d88e6 Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/159725289) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159593526",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159725289",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc01102e1595476cea82051bb992acb9427da257",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7357",
       "triggerID" : "bc01102e1595476cea82051bb992acb9427da257",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "612549149",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "612549149",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "612735854",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "612735854",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "613282091",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "613282091",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/160182477",
       "triggerID" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7449",
       "triggerID" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b7adab126357d7fb74426dcb9b5fb0b765d8eead Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/159866074) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343) 
   * 5110feb83332d8d0583a2be9534ac1565ef0be7b Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/160182477) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7449) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kl0u edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
kl0u edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-613340519
 
 
   @godfreyhe I just saw you last commit, I think it was a nice change to pass the `cacheFile` in the constructor of the `PlanGenerator` and register the files there. This delegates the whole process of the "plan creation" to the new class.
   
   My comment was just to remove the new public method :)

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159593526",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159725289",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc01102e1595476cea82051bb992acb9427da257",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7357",
       "triggerID" : "bc01102e1595476cea82051bb992acb9427da257",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "612549149",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "612549149",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "612735854",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * b7adab126357d7fb74426dcb9b5fb0b765d8eead Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/159866074) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159593526",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159725289",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc01102e1595476cea82051bb992acb9427da257",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7357",
       "triggerID" : "bc01102e1595476cea82051bb992acb9427da257",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "612549149",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "612549149",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "612735854",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "612735854",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "613282091",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "613282091",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/160182477",
       "triggerID" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7449",
       "triggerID" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b90f363254f04c75de526ddb0e3a2bed9686373",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2b90f363254f04c75de526ddb0e3a2bed9686373",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5110feb83332d8d0583a2be9534ac1565ef0be7b Travis: [CANCELED](https://travis-ci.com/github/flink-ci/flink/builds/160182477) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7449) 
   * 2b90f363254f04c75de526ddb0e3a2bed9686373 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159593526",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159725289",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc01102e1595476cea82051bb992acb9427da257",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7357",
       "triggerID" : "bc01102e1595476cea82051bb992acb9427da257",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "612549149",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "612549149",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "612735854",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "612735854",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "613282091",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "613282091",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/160182477",
       "triggerID" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7449",
       "triggerID" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b90f363254f04c75de526ddb0e3a2bed9686373",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2b90f363254f04c75de526ddb0e3a2bed9686373",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e2bccdeb54eb8066d91325213a8d30a1d292e96",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7e2bccdeb54eb8066d91325213a8d30a1d292e96",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5110feb83332d8d0583a2be9534ac1565ef0be7b Travis: [CANCELED](https://travis-ci.com/github/flink-ci/flink/builds/160182477) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7449) 
   * 2b90f363254f04c75de526ddb0e3a2bed9686373 UNKNOWN
   * 7e2bccdeb54eb8066d91325213a8d30a1d292e96 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159593526",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159725289",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f82a2583fb82c50376de97b616de11db678d88e6 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/159725289) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308) 
   * b7adab126357d7fb74426dcb9b5fb0b765d8eead Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/159866074) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] godfreyhe commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#discussion_r410028232
 
 

 ##########
 File path: flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/TestUtils.java
 ##########
 @@ -91,8 +91,6 @@ private TestUtils() {}
 	 * @param expectedParallelism expected operator parallelism
 	 */
 	public static void verifyParallelism(ExecutionEnvironment env, int expectedParallelism) {
-		env.setParallelism(2 * expectedParallelism);
 
 Review comment:
   hi @kl0u I have reverted this change. do you have other concern ?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] aljoscha commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
aljoscha commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#discussion_r406135868
 
 

 ##########
 File path: flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
 ##########
 @@ -1035,6 +1030,14 @@ protected void registerCachedFilesWithPlan(Plan p) throws IOException {
 		}
 	}
 
+	/**
 
 Review comment:
   True, but I think this is tricky, because there could be any number of options that we want to forward from the `ExecutionEnvironment` but that are easy to forget. If there is a general way of forwarding everything that would probably be ok.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] godfreyhe commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-615109554
 
 
   > Changes look good @godfreyhe ! I will merge as soon as my Travis gives green.
   
   Thanks you for your patient suggestion!

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kl0u commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
kl0u commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-615147044
 
 
   Thanks for the contribution @godfreyhe !

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kl0u commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
kl0u commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-613311020
 
 
   Hehe I see.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159593526",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159725289",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f82a2583fb82c50376de97b616de11db678d88e6 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/159725289) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b7e5ae168bcf572e6321257246a778951e5bb6a0 Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/159317260) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kl0u edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
kl0u edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-613340519
 
 
   @godfreyhe I just saw you last commit, I think it was a nice change to pass the `cacheFile` in the constructor of the `PlanGenerator` and register the files there. This delegates the whole process of the "plan creation" to the new class.
   
   My comment was just to remove the new public method in the `ExecutionEnvironemnt` :)

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] godfreyhe commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#discussion_r406122262
 
 

 ##########
 File path: flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
 ##########
 @@ -1035,6 +1030,14 @@ protected void registerCachedFilesWithPlan(Plan p) throws IOException {
 		}
 	}
 
+	/**
 
 Review comment:
   consider the following user table program:
   val env = ExecutionEnvironment.getExecutionEnvironment()
   val tableEnv = BatchTableEnvironment.create(env, config)
   env.registerCachedFile("/tmp/cache", "my_cache")
   val input = env.fromElements(...)...
   tEnv.fromDataSet(input)...
   tEnv.execute() // will generate plan through PlanGenerator in this method
   
   If we do not expose this method, the generated plan does not contain the cache files registered in DataSet program.(which can be converted to Table through `BatchTableEnvironment.fromDataSet` method)

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] godfreyhe commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#discussion_r408067969
 
 

 ##########
 File path: flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/TestUtils.java
 ##########
 @@ -91,8 +91,6 @@ private TestUtils() {}
 	 * @param expectedParallelism expected operator parallelism
 	 */
 	public static void verifyParallelism(ExecutionEnvironment env, int expectedParallelism) {
-		env.setParallelism(2 * expectedParallelism);
 
 Review comment:
   yes. before this pr, `CollectionEnvironment` overrides `getParallelism` method, and always return 1.  `ExecutionEnvironment#createProgramPlan` gets parallelism through `ExecutionEnvironment#getParallelism`. Now, `PlanGenerator` get parallelism through `config.getParallelism()`.  When `CollectionEnvironment` is created in `ExecutionEnvironment#createCollectionsEnvironment` method, the parallelism in config will be set as 1, while after `env.setParallelism(2 * expectedParallelism);` is called, the parallelism in config was changed to 4. which is unexpected.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kl0u commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
kl0u commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#discussion_r410041653
 
 

 ##########
 File path: flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/TestUtils.java
 ##########
 @@ -91,8 +91,6 @@ private TestUtils() {}
 	 * @param expectedParallelism expected operator parallelism
 	 */
 	public static void verifyParallelism(ExecutionEnvironment env, int expectedParallelism) {
-		env.setParallelism(2 * expectedParallelism);
 
 Review comment:
   @godfreyhe I agree that the propagation of the parallelism is confusing. But I think that we should not change existing behaviour in order to avoid users having unexpected results. I am having a look now at your PR.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] aljoscha commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
aljoscha commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#discussion_r406100037
 
 

 ##########
 File path: flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
 ##########
 @@ -1035,6 +1030,14 @@ protected void registerCachedFilesWithPlan(Plan p) throws IOException {
 		}
 	}
 
+	/**
 
 Review comment:
   This shouldn't be exposed. You can write the test strictly without using an `ExecutionEnvironment`. Which in the end users of `PlanGenerator` probably also have to do.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b7e5ae168bcf572e6321257246a778951e5bb6a0 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/159317260) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kl0u commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
kl0u commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-613271010
 
 
   Yes @godfreyhe , I will do it today. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159593526",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159725289",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc01102e1595476cea82051bb992acb9427da257",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7357",
       "triggerID" : "bc01102e1595476cea82051bb992acb9427da257",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * bc01102e1595476cea82051bb992acb9427da257 Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7357) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159593526",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159725289",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f82a2583fb82c50376de97b616de11db678d88e6 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/159725289) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308) 
   * b7adab126357d7fb74426dcb9b5fb0b765d8eead UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kl0u commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
kl0u commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-613340519
 
 
   @godfreyhe I just saw you last commit, I think it was a nice change to pass the `cacheFile` in the constructor of the `PlanGenerator` and register the files there. This delegates the whole process of the "plan creation" to the new class.
   
   My comment was just to not expose it through the new public method :)

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kl0u commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
kl0u commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#discussion_r407972646
 
 

 ##########
 File path: flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java
 ##########
 @@ -1035,6 +1030,14 @@ protected void registerCachedFilesWithPlan(Plan p) throws IOException {
 		}
 	}
 
+	/**
 
 Review comment:
   From the discussion, it seems that this method was added:
   1) for the tests 
   2) for future usage in the `TableEnvironment`.
   
   If this is correct, I would suggest to remove it from this PR and add it to the PR that uses it. Now it seems that this is an unused `public` method.
   
   In addition, I agree with @aljoscha 's concerns that we may need to come up with a better and more foolproof design on how to forward stuff from the `ExecutionEnvironment` to the `TableEnvironment`. So if we achieve that, then this method would be instant legacy. 
   
   WDYT?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610959224
 
 
   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit b7e5ae168bcf572e6321257246a778951e5bb6a0 (Wed Apr 08 13:27:46 UTC 2020)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
    * **This pull request references an unassigned [Jira ticket](https://issues.apache.org/jira/browse/FLINK-17052).** According to the [code contribution guide](https://flink.apache.org/contributing/contribute-code.html), tickets need to be assigned before starting with the implementation work.
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] godfreyhe commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#discussion_r408067969
 
 

 ##########
 File path: flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/TestUtils.java
 ##########
 @@ -91,8 +91,6 @@ private TestUtils() {}
 	 * @param expectedParallelism expected operator parallelism
 	 */
 	public static void verifyParallelism(ExecutionEnvironment env, int expectedParallelism) {
-		env.setParallelism(2 * expectedParallelism);
 
 Review comment:
   yes. before this pr, `CollectionEnvironment` overrides `getParallelism` method, and always return 1.  `ExecutionEnvironment#createProgramPlan` gets parallelism through `ExecutionEnvironment#getParallelism`. Now, `PlanGenerator` get parallelism through `config.getParallelism()`.  When `CollectionEnvironment` is created in `ExecutionEnvironment#createCollectionsEnvironment` method, the parallelism in config will be set as 1, while after `env.setParallelism(2 * expectedParallelism);` is called, the parallelism in config is changed to 4. that is unexpected.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] godfreyhe commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-613774692
 
 
   My private travis is green  https://travis-ci.org/github/godfreyhe/flink/builds/674785749

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kl0u commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
kl0u commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#discussion_r408061259
 
 

 ##########
 File path: flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/TestUtils.java
 ##########
 @@ -91,8 +91,6 @@ private TestUtils() {}
 	 * @param expectedParallelism expected operator parallelism
 	 */
 	public static void verifyParallelism(ExecutionEnvironment env, int expectedParallelism) {
-		env.setParallelism(2 * expectedParallelism);
 
 Review comment:
   Is this change needed?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159593526",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159725289",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc01102e1595476cea82051bb992acb9427da257",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bc01102e1595476cea82051bb992acb9427da257",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b7adab126357d7fb74426dcb9b5fb0b765d8eead Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/159866074) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343) 
   * bc01102e1595476cea82051bb992acb9427da257 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159593526",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159725289",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b7adab126357d7fb74426dcb9b5fb0b765d8eead Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/159866074) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159593526",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159725289",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc01102e1595476cea82051bb992acb9427da257",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7357",
       "triggerID" : "bc01102e1595476cea82051bb992acb9427da257",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "612549149",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "612549149",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * b7adab126357d7fb74426dcb9b5fb0b765d8eead Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/159866074) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kl0u commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
kl0u commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#discussion_r407999924
 
 

 ##########
 File path: flink-java/src/main/java/org/apache/flink/api/java/utils/PlanGenerator.java
 ##########
 @@ -0,0 +1,126 @@
+/*
+ * 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.java.utils;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.Plan;
+import org.apache.flink.api.common.operators.Operator;
+import org.apache.flink.api.common.operators.OperatorInformation;
+import org.apache.flink.api.java.operators.DataSink;
+import org.apache.flink.api.java.operators.OperatorTranslation;
+import org.apache.flink.api.java.typeutils.runtime.kryo.Serializers;
+import org.apache.flink.util.Visitor;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * A generator that generates a {@link Plan} from a graph of {@link Operator}s.
+ */
+public class PlanGenerator {
+	private static final Logger LOG = LoggerFactory.getLogger(PlanGenerator.class);
+
+	private final List<DataSink<?>> sinks;
+	private final ExecutionConfig config;
+	private final String jobName;
+
+	public PlanGenerator(
+			List<DataSink<?>> sinks,
+			ExecutionConfig config,
+			String jobName) {
+		this.sinks = sinks;
 
 Review comment:
   All these require `checkNotNull()` checks.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b7e5ae168bcf572e6321257246a778951e5bb6a0 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] godfreyhe commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-613282168
 
 
   @flinkbot run azure

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kl0u commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
kl0u commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#discussion_r408905254
 
 

 ##########
 File path: flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/TestUtils.java
 ##########
 @@ -91,8 +91,6 @@ private TestUtils() {}
 	 * @param expectedParallelism expected operator parallelism
 	 */
 	public static void verifyParallelism(ExecutionEnvironment env, int expectedParallelism) {
-		env.setParallelism(2 * expectedParallelism);
 
 Review comment:
   So given that users expect a specific behaviour, I think we should not change it, as this may break user code. WDYT?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159593526",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/159593526) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283) 
   * f82a2583fb82c50376de97b616de11db678d88e6 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159593526",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/159593526) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159593526",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159725289",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc01102e1595476cea82051bb992acb9427da257",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7357",
       "triggerID" : "bc01102e1595476cea82051bb992acb9427da257",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "612549149",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "612549149",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "612735854",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "612735854",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "613282091",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "613282091",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/160182477",
       "triggerID" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7449",
       "triggerID" : "5110feb83332d8d0583a2be9534ac1565ef0be7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b90f363254f04c75de526ddb0e3a2bed9686373",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2b90f363254f04c75de526ddb0e3a2bed9686373",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e2bccdeb54eb8066d91325213a8d30a1d292e96",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/160684351",
       "triggerID" : "7e2bccdeb54eb8066d91325213a8d30a1d292e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7e2bccdeb54eb8066d91325213a8d30a1d292e96",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7635",
       "triggerID" : "7e2bccdeb54eb8066d91325213a8d30a1d292e96",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5110feb83332d8d0583a2be9534ac1565ef0be7b Travis: [CANCELED](https://travis-ci.com/github/flink-ci/flink/builds/160182477) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7449) 
   * 2b90f363254f04c75de526ddb0e3a2bed9686373 UNKNOWN
   * 7e2bccdeb54eb8066d91325213a8d30a1d292e96 Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/160684351) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7635) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] godfreyhe commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#discussion_r408003774
 
 

 ##########
 File path: flink-java/src/main/java/org/apache/flink/api/java/utils/PlanGenerator.java
 ##########
 @@ -0,0 +1,126 @@
+/*
+ * 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.java.utils;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.Plan;
+import org.apache.flink.api.common.operators.Operator;
+import org.apache.flink.api.common.operators.OperatorInformation;
+import org.apache.flink.api.java.operators.DataSink;
+import org.apache.flink.api.java.operators.OperatorTranslation;
+import org.apache.flink.api.java.typeutils.runtime.kryo.Serializers;
+import org.apache.flink.util.Visitor;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * A generator that generates a {@link Plan} from a graph of {@link Operator}s.
+ */
+public class PlanGenerator {
+	private static final Logger LOG = LoggerFactory.getLogger(PlanGenerator.class);
+
+	private final List<DataSink<?>> sinks;
+	private final ExecutionConfig config;
+	private final String jobName;
+
+	public PlanGenerator(
+			List<DataSink<?>> sinks,
+			ExecutionConfig config,
+			String jobName) {
+		this.sinks = sinks;
+		this.config = config;
+		this.jobName = jobName;
+	}
+
+	public Plan generate() {
+		OperatorTranslation translator = new OperatorTranslation();
+		Plan plan = translator.translateToPlan(sinks, jobName);
+
+		if (config.getParallelism() > 0) {
+			plan.setDefaultParallelism(config.getParallelism());
+		}
+		plan.setExecutionConfig(config);
+
+		// Check plan for GenericTypeInfo's and register the types at the serializers.
+		if (!config.isAutoTypeRegistrationDisabled()) {
+			plan.accept(new Visitor<Operator<?>>() {
+
+				private final Set<Class<?>> registeredTypes = new HashSet<>();
+				private final Set<org.apache.flink.api.common.operators.Operator<?>> visitedOperators = new HashSet<>();
+
+				@Override
+				public boolean preVisit(org.apache.flink.api.common.operators.Operator<?> visitable) {
+					if (!visitedOperators.add(visitable)) {
+						return false;
+					}
+					OperatorInformation<?> opInfo = visitable.getOperatorInfo();
+					Serializers.recursivelyRegisterType(opInfo.getOutputType(), config, registeredTypes);
+					return true;
+				}
+
+				@Override
+				public void postVisit(org.apache.flink.api.common.operators.Operator<?> visitable) {
+				}
+			});
+		}
+
+		// All types are registered now. Print information.
+		int registeredTypes = config.getRegisteredKryoTypes().size() +
+				config.getRegisteredPojoTypes().size() +
+				config.getRegisteredTypesWithKryoSerializerClasses().size() +
+				config.getRegisteredTypesWithKryoSerializers().size();
+		int defaultKryoSerializers = config.getDefaultKryoSerializers().size() +
+				config.getDefaultKryoSerializerClasses().size();
+		LOG.info("The job has {} registered types and {} default Kryo serializers", registeredTypes,
+				defaultKryoSerializers);
+
+		if (config.isForceKryoEnabled() && config.isForceAvroEnabled()) {
+			LOG.warn("In the ExecutionConfig, both Avro and Kryo are enforced. Using Kryo serializer");
+		}
+		if (config.isForceKryoEnabled()) {
+			LOG.info("Using KryoSerializer for serializing POJOs");
+		}
+		if (config.isForceAvroEnabled()) {
+			LOG.info("Using AvroSerializer for serializing POJOs");
+		}
 
 Review comment:
   Thank for detailed check, I just copy this from `ExecutionEnvironment#createProgramPlan` method, I will correct this. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] godfreyhe commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#discussion_r409272606
 
 

 ##########
 File path: flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/generator/TestUtils.java
 ##########
 @@ -91,8 +91,6 @@ private TestUtils() {}
 	 * @param expectedParallelism expected operator parallelism
 	 */
 	public static void verifyParallelism(ExecutionEnvironment env, int expectedParallelism) {
-		env.setParallelism(2 * expectedParallelism);
 
 Review comment:
   One approach is we add a constructor parameter named `defaultParallelism` for `PlanGenerator`, give the value from `ExecutionEnvironment#getParallelism`. That does not  brake current behavior, but that also means the parallelism from `ExecutionEnvironment` and from `ExecutionConfig` may be different. WDYT ?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159593526",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159725289",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc01102e1595476cea82051bb992acb9427da257",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7357",
       "triggerID" : "bc01102e1595476cea82051bb992acb9427da257",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "612549149",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * bc01102e1595476cea82051bb992acb9427da257 Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7357) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kl0u commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
kl0u commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#discussion_r407995415
 
 

 ##########
 File path: flink-java/src/main/java/org/apache/flink/api/java/utils/PlanGenerator.java
 ##########
 @@ -0,0 +1,126 @@
+/*
+ * 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.java.utils;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.Plan;
+import org.apache.flink.api.common.operators.Operator;
+import org.apache.flink.api.common.operators.OperatorInformation;
+import org.apache.flink.api.java.operators.DataSink;
+import org.apache.flink.api.java.operators.OperatorTranslation;
+import org.apache.flink.api.java.typeutils.runtime.kryo.Serializers;
+import org.apache.flink.util.Visitor;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * A generator that generates a {@link Plan} from a graph of {@link Operator}s.
+ */
+public class PlanGenerator {
+	private static final Logger LOG = LoggerFactory.getLogger(PlanGenerator.class);
+
+	private final List<DataSink<?>> sinks;
+	private final ExecutionConfig config;
+	private final String jobName;
+
+	public PlanGenerator(
+			List<DataSink<?>> sinks,
+			ExecutionConfig config,
+			String jobName) {
+		this.sinks = sinks;
+		this.config = config;
+		this.jobName = jobName;
+	}
+
+	public Plan generate() {
+		OperatorTranslation translator = new OperatorTranslation();
+		Plan plan = translator.translateToPlan(sinks, jobName);
+
+		if (config.getParallelism() > 0) {
+			plan.setDefaultParallelism(config.getParallelism());
+		}
+		plan.setExecutionConfig(config);
 
 Review comment:
   This can go in a method:
   
   ```
   private Plan createPlan() {
   		final OperatorTranslation translator = new OperatorTranslation();
   		final Plan plan = translator.translateToPlan(sinks, jobName);
   
   		if (config.getParallelism() > 0) {
   			plan.setDefaultParallelism(config.getParallelism());
   		}
   		plan.setExecutionConfig(config);
   		return plan;
   	}
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159593526",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159725289",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc01102e1595476cea82051bb992acb9427da257",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7357",
       "triggerID" : "bc01102e1595476cea82051bb992acb9427da257",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "612549149",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "612549149",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "612735854",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "612735854",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "613282091",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "613282091",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * b7adab126357d7fb74426dcb9b5fb0b765d8eead Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/159866074) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] kl0u commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
kl0u commented on a change in pull request #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#discussion_r407992545
 
 

 ##########
 File path: flink-java/src/main/java/org/apache/flink/api/java/utils/PlanGenerator.java
 ##########
 @@ -0,0 +1,126 @@
+/*
+ * 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.java.utils;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.Plan;
+import org.apache.flink.api.common.operators.Operator;
+import org.apache.flink.api.common.operators.OperatorInformation;
+import org.apache.flink.api.java.operators.DataSink;
+import org.apache.flink.api.java.operators.OperatorTranslation;
+import org.apache.flink.api.java.typeutils.runtime.kryo.Serializers;
+import org.apache.flink.util.Visitor;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * A generator that generates a {@link Plan} from a graph of {@link Operator}s.
+ */
+public class PlanGenerator {
+	private static final Logger LOG = LoggerFactory.getLogger(PlanGenerator.class);
+
+	private final List<DataSink<?>> sinks;
+	private final ExecutionConfig config;
+	private final String jobName;
+
+	public PlanGenerator(
+			List<DataSink<?>> sinks,
+			ExecutionConfig config,
+			String jobName) {
+		this.sinks = sinks;
+		this.config = config;
+		this.jobName = jobName;
+	}
+
+	public Plan generate() {
+		OperatorTranslation translator = new OperatorTranslation();
+		Plan plan = translator.translateToPlan(sinks, jobName);
+
+		if (config.getParallelism() > 0) {
+			plan.setDefaultParallelism(config.getParallelism());
+		}
+		plan.setExecutionConfig(config);
+
+		// Check plan for GenericTypeInfo's and register the types at the serializers.
+		if (!config.isAutoTypeRegistrationDisabled()) {
+			plan.accept(new Visitor<Operator<?>>() {
+
+				private final Set<Class<?>> registeredTypes = new HashSet<>();
+				private final Set<org.apache.flink.api.common.operators.Operator<?>> visitedOperators = new HashSet<>();
+
+				@Override
+				public boolean preVisit(org.apache.flink.api.common.operators.Operator<?> visitable) {
+					if (!visitedOperators.add(visitable)) {
+						return false;
+					}
+					OperatorInformation<?> opInfo = visitable.getOperatorInfo();
+					Serializers.recursivelyRegisterType(opInfo.getOutputType(), config, registeredTypes);
+					return true;
+				}
+
+				@Override
+				public void postVisit(org.apache.flink.api.common.operators.Operator<?> visitable) {
+				}
+			});
+		}
+
+		// All types are registered now. Print information.
+		int registeredTypes = config.getRegisteredKryoTypes().size() +
+				config.getRegisteredPojoTypes().size() +
+				config.getRegisteredTypesWithKryoSerializerClasses().size() +
+				config.getRegisteredTypesWithKryoSerializers().size();
+		int defaultKryoSerializers = config.getDefaultKryoSerializers().size() +
+				config.getDefaultKryoSerializerClasses().size();
+		LOG.info("The job has {} registered types and {} default Kryo serializers", registeredTypes,
+				defaultKryoSerializers);
+
+		if (config.isForceKryoEnabled() && config.isForceAvroEnabled()) {
+			LOG.warn("In the ExecutionConfig, both Avro and Kryo are enforced. Using Kryo serializer");
+		}
+		if (config.isForceKryoEnabled()) {
+			LOG.info("Using KryoSerializer for serializing POJOs");
+		}
+		if (config.isForceAvroEnabled()) {
+			LOG.info("Using AvroSerializer for serializing POJOs");
+		}
 
 Review comment:
   This block here seems a bit weird, right? It may end up printing:
   ```
   LOG.warn("In the ExecutionConfig, both Avro and Kryo are enforced. Using Kryo serializer");
   LOG.info("Using KryoSerializer for serializing POJOs");
   LOG.info("Using AvroSerializer for serializing POJOs");
   ```
   which is misleading as either `kryo` or `avro` will be used.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] godfreyhe edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
godfreyhe edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-615109554
 
 
   > Changes look good @godfreyhe ! I will merge as soon as my Travis gives green.
   
   Thanks for your patient suggestion!

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11671: [FLINK-17052] [client] Introduce PlanGenerator
URL: https://github.com/apache/flink/pull/11671#issuecomment-610971753
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7230",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159317260",
       "triggerID" : "b7e5ae168bcf572e6321257246a778951e5bb6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7283",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159593526",
       "triggerID" : "0bb3e20d6406a3db0a0d735d2b78f77cadd7c59b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159725289",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7308",
       "triggerID" : "f82a2583fb82c50376de97b616de11db678d88e6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/159866074",
       "triggerID" : "b7adab126357d7fb74426dcb9b5fb0b765d8eead",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bc01102e1595476cea82051bb992acb9427da257",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7357",
       "triggerID" : "bc01102e1595476cea82051bb992acb9427da257",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b7adab126357d7fb74426dcb9b5fb0b765d8eead Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/159866074) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7343) 
   * bc01102e1595476cea82051bb992acb9427da257 Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7357) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services