From b2f8db5ea59c4e927e684a733f55b4e816a72b07 Mon Sep 17 00:00:00 2001 From: yuzhongliu Date: Mon, 29 Aug 2016 19:03:15 +0800 Subject: [PATCH 001/299] #4450 update storm version to 1.0.0 --- flink-contrib/flink-storm/pom.xml | 8 +- .../apache/flink/storm/api/FlinkClient.java | 20 +- .../flink/storm/api/FlinkLocalCluster.java | 15 +- .../storm/api/FlinkOutputFieldsDeclarer.java | 6 +- .../flink/storm/api/FlinkSubmitter.java | 17 +- .../apache/flink/storm/api/FlinkTopology.java | 18 +- .../storm/api/StormFlinkStreamMerger.java | 6 +- .../storm/api/TwoFlinkStreamsMerger.java | 6 +- .../apache/flink/storm/util/FiniteSpout.java | 2 +- .../storm/util/NullTerminatingSpout.java | 8 +- .../util/SpoutOutputCollectorObserver.java | 6 +- .../apache/flink/storm/util/StormConfig.java | 244 +++++++++--------- .../flink/storm/wrappers/BoltCollector.java | 7 +- .../flink/storm/wrappers/BoltWrapper.java | 18 +- .../storm/wrappers/FlinkTopologyContext.java | 20 +- .../wrappers/MergedInputsBoltWrapper.java | 2 +- .../wrappers/SetupOutputFieldsDeclarer.java | 6 +- .../flink/storm/wrappers/SpoutCollector.java | 7 +- .../flink/storm/wrappers/SpoutWrapper.java | 8 +- .../flink/storm/wrappers/StormTuple.java | 18 +- .../storm/wrappers/WrapperSetupHelper.java | 24 +- .../api/FlinkOutputFieldsDeclarerTest.java | 4 +- .../flink/storm/api/FlinkTopologyTest.java | 4 +- .../org/apache/flink/storm/api/TestBolt.java | 10 +- .../org/apache/flink/storm/api/TestSpout.java | 8 +- .../flink/storm/util/FiniteTestSpout.java | 12 +- .../storm/util/NullTerminatingSpoutTest.java | 8 +- .../SpoutOutputCollectorObserverTest.java | 5 +- .../flink/storm/util/TestDummyBolt.java | 14 +- .../flink/storm/util/TestDummySpout.java | 14 +- .../org/apache/flink/storm/util/TestSink.java | 10 +- .../storm/wrappers/BoltCollectorTest.java | 2 +- .../flink/storm/wrappers/BoltWrapperTest.java | 18 +- .../wrappers/FlinkTopologyContextTest.java | 14 +- .../SetupOutputFieldsDeclarerTest.java | 4 +- .../storm/wrappers/SpoutCollectorTest.java | 2 +- .../storm/wrappers/SpoutWrapperTest.java | 8 +- .../flink/storm/wrappers/StormTupleTest.java | 8 +- .../wrappers/WrapperSetupHelperTest.java | 32 +-- 39 files changed, 327 insertions(+), 316 deletions(-) diff --git a/flink-contrib/flink-storm/pom.xml b/flink-contrib/flink-storm/pom.xml index 0467fad822848..d5def18305ecf 100644 --- a/flink-contrib/flink-storm/pom.xml +++ b/flink-contrib/flink-storm/pom.xml @@ -71,7 +71,7 @@ under the License. org.apache.storm storm-core - 0.9.4 + 1.0.0 org.slf4j @@ -139,6 +139,12 @@ under the License. + + com.googlecode.json-simple + json-simple + 1.1 + compile + diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java index f4bcfb711c5f8..46f8a7528cd72 100644 --- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkClient.java @@ -23,14 +23,14 @@ import akka.pattern.Patterns; import akka.util.Timeout; -import backtype.storm.Config; -import backtype.storm.generated.AlreadyAliveException; -import backtype.storm.generated.InvalidTopologyException; -import backtype.storm.generated.KillOptions; -import backtype.storm.generated.Nimbus; -import backtype.storm.generated.NotAliveException; -import backtype.storm.utils.NimbusClient; -import backtype.storm.utils.Utils; +import org.apache.storm.Config; +import org.apache.storm.generated.AlreadyAliveException; +import org.apache.storm.generated.InvalidTopologyException; +import org.apache.storm.generated.KillOptions; +import org.apache.storm.generated.Nimbus; +import org.apache.storm.generated.NotAliveException; +import org.apache.storm.utils.NimbusClient; +import org.apache.storm.utils.Utils; import com.esotericsoftware.kryo.Serializer; @@ -89,7 +89,7 @@ public class FlinkClient { /** The user specified timeout in milliseconds */ private final String timeout; - // The following methods are derived from "backtype.storm.utils.NimbusClient" + // The following methods are derived from "org.apache.storm.utils.NimbusClient" /** * Instantiates a new {@link FlinkClient} for the given configuration, host name, and port. If values for {@link @@ -158,7 +158,7 @@ public FlinkClient getClient() { return this; } - // The following methods are derived from "backtype.storm.generated.Nimubs.Client" + // The following methods are derived from "org.apache.storm.generated.Nimubs.Client" /** * Parameter {@code uploadedJarLocation} is actually used to point to the local jar, because Flink does not support diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkLocalCluster.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkLocalCluster.java index da19a22193a87..6022bb5a5e19a 100644 --- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkLocalCluster.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkLocalCluster.java @@ -18,20 +18,19 @@ package org.apache.flink.storm.api; -import backtype.storm.LocalCluster; -import backtype.storm.generated.ClusterSummary; -import backtype.storm.generated.KillOptions; -import backtype.storm.generated.RebalanceOptions; -import backtype.storm.generated.StormTopology; -import backtype.storm.generated.SubmitOptions; -import backtype.storm.generated.TopologyInfo; - import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.minicluster.FlinkMiniCluster; import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.streaming.api.graph.StreamGraph; +import org.apache.storm.LocalCluster; +import org.apache.storm.generated.SubmitOptions; +import org.apache.storm.generated.KillOptions; +import org.apache.storm.generated.RebalanceOptions; +import org.apache.storm.generated.ClusterSummary; +import org.apache.storm.generated.StormTopology; +import org.apache.storm.generated.TopologyInfo; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkOutputFieldsDeclarer.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkOutputFieldsDeclarer.java index 794beee8ab9bc..f28c7b47e545f 100644 --- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkOutputFieldsDeclarer.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkOutputFieldsDeclarer.java @@ -17,9 +17,9 @@ package org.apache.flink.storm.api; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Fields; -import backtype.storm.utils.Utils; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.tuple.Fields; +import org.apache.storm.utils.Utils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.typeutils.TypeExtractor; diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkSubmitter.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkSubmitter.java index f8932b126d3d6..87b48933eae1c 100644 --- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkSubmitter.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkSubmitter.java @@ -16,25 +16,24 @@ */ package org.apache.flink.storm.api; -import backtype.storm.Config; -import backtype.storm.StormSubmitter; -import backtype.storm.generated.AlreadyAliveException; -import backtype.storm.generated.InvalidTopologyException; -import backtype.storm.generated.SubmitOptions; -import backtype.storm.utils.Utils; - -import java.net.URISyntaxException; -import java.net.URL; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.client.program.ContextEnvironment; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.storm.Config; +import org.apache.storm.StormSubmitter; +import org.apache.storm.generated.AlreadyAliveException; +import org.apache.storm.generated.InvalidTopologyException; +import org.apache.storm.generated.SubmitOptions; +import org.apache.storm.utils.Utils; import org.json.simple.JSONValue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.File; +import java.net.URISyntaxException; +import java.net.URL; import java.util.Map; /** diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopology.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopology.java index 2546f1752babd..2b36feb63cd94 100644 --- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopology.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopology.java @@ -18,15 +18,15 @@ */ package org.apache.flink.storm.api; -import backtype.storm.generated.ComponentCommon; -import backtype.storm.generated.GlobalStreamId; -import backtype.storm.generated.Grouping; -import backtype.storm.generated.StormTopology; -import backtype.storm.topology.IRichBolt; -import backtype.storm.topology.IRichSpout; -import backtype.storm.topology.IRichStateSpout; -import backtype.storm.topology.TopologyBuilder; -import backtype.storm.tuple.Fields; +import org.apache.storm.generated.ComponentCommon; +import org.apache.storm.generated.GlobalStreamId; +import org.apache.storm.generated.Grouping; +import org.apache.storm.generated.StormTopology; +import org.apache.storm.topology.IRichBolt; +import org.apache.storm.topology.IRichSpout; +import org.apache.storm.topology.IRichStateSpout; +import org.apache.storm.topology.TopologyBuilder; +import org.apache.storm.tuple.Fields; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.typeinfo.TypeInformation; diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/StormFlinkStreamMerger.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/StormFlinkStreamMerger.java index 72c156970e6a7..160a7d9ea703b 100644 --- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/StormFlinkStreamMerger.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/StormFlinkStreamMerger.java @@ -16,9 +16,9 @@ import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction; import org.apache.flink.util.Collector; -import backtype.storm.generated.GlobalStreamId; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.MessageId; +import org.apache.storm.generated.GlobalStreamId; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.MessageId; /** * Merges a stream of type {@link StormTuple} with a Flink {@link DataStreams} into a stream of type {@link StormTuple}. diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/TwoFlinkStreamsMerger.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/TwoFlinkStreamsMerger.java index 1866e325b6fff..1e6e2ed3b5aa7 100644 --- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/TwoFlinkStreamsMerger.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/TwoFlinkStreamsMerger.java @@ -16,9 +16,9 @@ import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction; import org.apache.flink.util.Collector; -import backtype.storm.generated.GlobalStreamId; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.MessageId; +import org.apache.storm.generated.GlobalStreamId; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.MessageId; /** * Merges two Flink {@link DataStreams} into a stream of type {@link StormTuple}. diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/FiniteSpout.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/FiniteSpout.java index 99c258387c161..10f9797baa427 100644 --- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/FiniteSpout.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/FiniteSpout.java @@ -17,7 +17,7 @@ package org.apache.flink.storm.util; -import backtype.storm.topology.IRichSpout; +import org.apache.storm.topology.IRichSpout; /** * This interface represents a spout that emits a finite number of records. Common spouts emit infinite streams by diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/NullTerminatingSpout.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/NullTerminatingSpout.java index 23d9d709aa278..20e3309590cfd 100644 --- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/NullTerminatingSpout.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/NullTerminatingSpout.java @@ -19,10 +19,10 @@ import java.util.Map; -import backtype.storm.spout.SpoutOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IRichSpout; -import backtype.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.spout.SpoutOutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.IRichSpout; +import org.apache.storm.topology.OutputFieldsDeclarer; /** * {@link NullTerminatingSpout} in a finite spout (ie, implements {@link FiniteSpout} interface) that wraps an diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SpoutOutputCollectorObserver.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SpoutOutputCollectorObserver.java index b79cc4e13fae2..5c6c20d27733d 100644 --- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SpoutOutputCollectorObserver.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SpoutOutputCollectorObserver.java @@ -17,10 +17,10 @@ */ package org.apache.flink.storm.util; -import java.util.List; +import org.apache.storm.spout.SpoutOutputCollector; +import org.apache.storm.utils.Utils; -import backtype.storm.spout.SpoutOutputCollector; -import backtype.storm.utils.Utils; +import java.util.List; /** * Observes if a call to any {@code emit(...)} or {@code emitDirect(...)} method is made. diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/StormConfig.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/StormConfig.java index 38ce58c719f4d..040c395936dd5 100644 --- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/StormConfig.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/StormConfig.java @@ -1,122 +1,122 @@ -/* - * 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.storm.util; - -import backtype.storm.Config; -import org.apache.flink.api.common.ExecutionConfig.GlobalJobParameters; - -import java.util.Collection; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; - -/** - * {@link StormConfig} is used to provide a user-defined Storm configuration (ie, a raw {@link Map} or {@link Config} - * object) for embedded Spouts and Bolts. - */ -@SuppressWarnings("rawtypes") -public final class StormConfig extends GlobalJobParameters implements Map { - private static final long serialVersionUID = 8019519109673698490L; - - /** Contains the actual configuration that is provided to Spouts and Bolts. */ - private final Map config = new HashMap(); - - /** - * Creates an empty configuration. - */ - public StormConfig() { - } - - /** - * Creates an configuration with initial values provided by the given {@code Map}. - * - * @param config - * Initial values for this configuration. - */ - @SuppressWarnings("unchecked") - public StormConfig(Map config) { - this.config.putAll(config); - } - - - @Override - public int size() { - return this.config.size(); - } - - @Override - public boolean isEmpty() { - return this.config.isEmpty(); - } - - @Override - public boolean containsKey(Object key) { - return this.config.containsKey(key); - } - - @Override - public boolean containsValue(Object value) { - return this.config.containsValue(value); - } - - @Override - public Object get(Object key) { - return this.config.get(key); - } - - @SuppressWarnings("unchecked") - @Override - public Object put(Object key, Object value) { - return this.config.put(key, value); - } - - @Override - public Object remove(Object key) { - return this.config.remove(key); - } - - @SuppressWarnings("unchecked") - @Override - public void putAll(Map m) { - this.config.putAll(m); - } - - @Override - public void clear() { - this.config.clear(); - } - - @SuppressWarnings("unchecked") - @Override - public Set keySet() { - return this.config.keySet(); - } - - @SuppressWarnings("unchecked") - @Override - public Collection values() { - return this.config.values(); - } - - @SuppressWarnings("unchecked") - @Override - public Set> entrySet() { - return this.config.entrySet(); - } - -} +/* + * 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.storm.util; + +import org.apache.storm.Config; +import org.apache.flink.api.common.ExecutionConfig.GlobalJobParameters; + +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +/** + * {@link StormConfig} is used to provide a user-defined Storm configuration (ie, a raw {@link Map} or {@link Config} + * object) for embedded Spouts and Bolts. + */ +@SuppressWarnings("rawtypes") +public final class StormConfig extends GlobalJobParameters implements Map { + private static final long serialVersionUID = 8019519109673698490L; + + /** Contains the actual configuration that is provided to Spouts and Bolts. */ + private final Map config = new HashMap(); + + /** + * Creates an empty configuration. + */ + public StormConfig() { + } + + /** + * Creates an configuration with initial values provided by the given {@code Map}. + * + * @param config + * Initial values for this configuration. + */ + @SuppressWarnings("unchecked") + public StormConfig(Map config) { + this.config.putAll(config); + } + + + @Override + public int size() { + return this.config.size(); + } + + @Override + public boolean isEmpty() { + return this.config.isEmpty(); + } + + @Override + public boolean containsKey(Object key) { + return this.config.containsKey(key); + } + + @Override + public boolean containsValue(Object value) { + return this.config.containsValue(value); + } + + @Override + public Object get(Object key) { + return this.config.get(key); + } + + @SuppressWarnings("unchecked") + @Override + public Object put(Object key, Object value) { + return this.config.put(key, value); + } + + @Override + public Object remove(Object key) { + return this.config.remove(key); + } + + @SuppressWarnings("unchecked") + @Override + public void putAll(Map m) { + this.config.putAll(m); + } + + @Override + public void clear() { + this.config.clear(); + } + + @SuppressWarnings("unchecked") + @Override + public Set keySet() { + return this.config.keySet(); + } + + @SuppressWarnings("unchecked") + @Override + public Collection values() { + return this.config.values(); + } + + @SuppressWarnings("unchecked") + @Override + public Set> entrySet() { + return this.config.entrySet(); + } + +} diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltCollector.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltCollector.java index 2196a1cbb90aa..22daa54d0f4a1 100644 --- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltCollector.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltCollector.java @@ -17,8 +17,8 @@ package org.apache.flink.storm.wrappers; -import backtype.storm.task.IOutputCollector; -import backtype.storm.tuple.Tuple; +import org.apache.storm.task.IOutputCollector; +import org.apache.storm.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple0; import org.apache.flink.api.java.tuple.Tuple25; @@ -88,4 +88,7 @@ public void ack(final Tuple input) {} @Override public void fail(final Tuple input) {} + @Override + public void resetTimeout(final Tuple input) {} + } diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltWrapper.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltWrapper.java index d59ff04027b50..113c2a9ed8ae9 100644 --- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltWrapper.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltWrapper.java @@ -17,15 +17,15 @@ package org.apache.flink.storm.wrappers; -import backtype.storm.generated.GlobalStreamId; -import backtype.storm.generated.Grouping; -import backtype.storm.generated.StormTopology; -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IRichBolt; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.MessageId; -import backtype.storm.utils.Utils; +import org.apache.storm.generated.GlobalStreamId; +import org.apache.storm.generated.Grouping; +import org.apache.storm.generated.StormTopology; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.IRichBolt; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.MessageId; +import org.apache.storm.utils.Utils; import org.apache.flink.api.common.ExecutionConfig.GlobalJobParameters; import org.apache.flink.api.java.tuple.Tuple; diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/FlinkTopologyContext.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/FlinkTopologyContext.java index 52d39a7e762f7..f55f0e33a9f95 100644 --- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/FlinkTopologyContext.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/FlinkTopologyContext.java @@ -17,16 +17,16 @@ package org.apache.flink.storm.wrappers; -import backtype.storm.generated.StormTopology; -import backtype.storm.hooks.ITaskHook; -import backtype.storm.metric.api.CombinedMetric; -import backtype.storm.metric.api.ICombiner; -import backtype.storm.metric.api.IMetric; -import backtype.storm.metric.api.IReducer; -import backtype.storm.metric.api.ReducedMetric; -import backtype.storm.state.ISubscribedState; -import backtype.storm.task.TopologyContext; -import backtype.storm.tuple.Fields; +import org.apache.storm.generated.StormTopology; +import org.apache.storm.hooks.ITaskHook; +import org.apache.storm.metric.api.CombinedMetric; +import org.apache.storm.metric.api.ICombiner; +import org.apache.storm.metric.api.IMetric; +import org.apache.storm.metric.api.IReducer; +import org.apache.storm.metric.api.ReducedMetric; +import org.apache.storm.state.ISubscribedState; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.tuple.Fields; import clojure.lang.Atom; import java.util.Collection; diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/MergedInputsBoltWrapper.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/MergedInputsBoltWrapper.java index 7a3b6d5b54183..6dd6973691a1b 100644 --- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/MergedInputsBoltWrapper.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/MergedInputsBoltWrapper.java @@ -17,7 +17,7 @@ package org.apache.flink.storm.wrappers; -import backtype.storm.topology.IRichBolt; +import org.apache.storm.topology.IRichBolt; import org.apache.flink.api.java.tuple.Tuple0; import org.apache.flink.api.java.tuple.Tuple1; diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SetupOutputFieldsDeclarer.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SetupOutputFieldsDeclarer.java index daf925211fdbb..d927f0ef65243 100644 --- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SetupOutputFieldsDeclarer.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SetupOutputFieldsDeclarer.java @@ -17,9 +17,9 @@ package org.apache.flink.storm.wrappers; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Fields; -import backtype.storm.utils.Utils; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.tuple.Fields; +import org.apache.storm.utils.Utils; import java.util.HashMap; diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutCollector.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutCollector.java index 0e2190e29b1cd..c1389bc1b42e1 100644 --- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutCollector.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutCollector.java @@ -17,7 +17,7 @@ package org.apache.flink.storm.wrappers; -import backtype.storm.spout.ISpoutOutputCollector; +import org.apache.storm.spout.ISpoutOutputCollector; import org.apache.flink.api.java.tuple.Tuple0; import org.apache.flink.api.java.tuple.Tuple25; import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext; @@ -79,4 +79,9 @@ public void emitDirect(final int taskId, final String streamId, final List implements backtype.storm.tuple.Tuple { +public class StormTuple implements org.apache.storm.tuple.Tuple { /** The Storm representation of the original Flink tuple. */ private final Values stormTuple; @@ -389,4 +389,10 @@ public String toString() { return "StormTuple{ " + stormTuple.toString() + "[" + this.producerComponentId + "," + this.producerStreamId + "," + this.producerTaskId + "," + this.messageId + "]}"; } + + @Override + @Deprecated + public GlobalStreamId getSourceGlobalStreamId() { + return new GlobalStreamId(this.producerComponentId, this.producerStreamId); + } } diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/WrapperSetupHelper.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/WrapperSetupHelper.java index 74a12ddcab696..3a9b650d792a3 100644 --- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/WrapperSetupHelper.java +++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/WrapperSetupHelper.java @@ -16,18 +16,18 @@ */ package org.apache.flink.storm.wrappers; -import backtype.storm.Config; -import backtype.storm.generated.Bolt; -import backtype.storm.generated.ComponentCommon; -import backtype.storm.generated.SpoutSpec; -import backtype.storm.generated.StateSpoutSpec; -import backtype.storm.generated.StormTopology; -import backtype.storm.generated.StreamInfo; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IComponent; -import backtype.storm.topology.IRichBolt; -import backtype.storm.topology.IRichSpout; -import backtype.storm.tuple.Fields; +import org.apache.storm.Config; +import org.apache.storm.generated.Bolt; +import org.apache.storm.generated.ComponentCommon; +import org.apache.storm.generated.SpoutSpec; +import org.apache.storm.generated.StateSpoutSpec; +import org.apache.storm.generated.StormTopology; +import org.apache.storm.generated.StreamInfo; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.IComponent; +import org.apache.storm.topology.IRichBolt; +import org.apache.storm.topology.IRichSpout; +import org.apache.storm.tuple.Fields; import clojure.lang.Atom; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkOutputFieldsDeclarerTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkOutputFieldsDeclarerTest.java index 90a82ba2fd415..ddbeaff964567 100644 --- a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkOutputFieldsDeclarerTest.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkOutputFieldsDeclarerTest.java @@ -16,8 +16,8 @@ */ package org.apache.flink.storm.api; -import backtype.storm.tuple.Fields; -import backtype.storm.utils.Utils; +import org.apache.storm.tuple.Fields; +import org.apache.storm.utils.Utils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.storm.util.AbstractTest; import org.junit.Assert; diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyTest.java index 39b01d849a0df..0ec0179ebefe9 100644 --- a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyTest.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyTest.java @@ -16,8 +16,8 @@ */ package org.apache.flink.storm.api; -import backtype.storm.topology.TopologyBuilder; -import backtype.storm.tuple.Fields; +import org.apache.storm.topology.TopologyBuilder; +import org.apache.storm.tuple.Fields; import org.apache.flink.storm.util.TestDummyBolt; import org.apache.flink.storm.util.TestDummySpout; import org.apache.flink.storm.util.TestSink; diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestBolt.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestBolt.java index 6077534f9082c..0f617fb766a86 100644 --- a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestBolt.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestBolt.java @@ -16,11 +16,11 @@ */ package org.apache.flink.storm.api; -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IRichBolt; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Tuple; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.IRichBolt; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.tuple.Tuple; import java.util.Map; diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestSpout.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestSpout.java index 846ae5102f4fe..1b185a7ada99f 100644 --- a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestSpout.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/TestSpout.java @@ -16,10 +16,10 @@ */ package org.apache.flink.storm.api; -import backtype.storm.spout.SpoutOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IRichSpout; -import backtype.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.spout.SpoutOutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.IRichSpout; +import org.apache.storm.topology.OutputFieldsDeclarer; import java.util.Map; diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/FiniteTestSpout.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/FiniteTestSpout.java index 1b320e5a642d1..9a5b1cd7a0236 100644 --- a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/FiniteTestSpout.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/FiniteTestSpout.java @@ -17,12 +17,12 @@ package org.apache.flink.storm.util; -import backtype.storm.spout.SpoutOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IRichSpout; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Values; +import org.apache.storm.spout.SpoutOutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.IRichSpout; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Values; import java.util.Map; diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/NullTerminatingSpoutTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/NullTerminatingSpoutTest.java index da2021cf446e0..1eaed4aebf74a 100644 --- a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/NullTerminatingSpoutTest.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/NullTerminatingSpoutTest.java @@ -20,10 +20,10 @@ import java.util.HashMap; import java.util.Map; -import backtype.storm.spout.SpoutOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IRichSpout; -import backtype.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.spout.SpoutOutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.IRichSpout; +import org.apache.storm.topology.OutputFieldsDeclarer; import org.junit.Assert; import org.junit.Test; diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/SpoutOutputCollectorObserverTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/SpoutOutputCollectorObserverTest.java index 0e3784a320424..3d43a00edcc69 100644 --- a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/SpoutOutputCollectorObserverTest.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/SpoutOutputCollectorObserverTest.java @@ -17,14 +17,15 @@ */ package org.apache.flink.storm.util; -import backtype.storm.spout.SpoutOutputCollector; +import org.apache.storm.spout.SpoutOutputCollector; import org.junit.Assert; import org.junit.Test; import static org.mockito.Mockito.mock; -public class SpoutOutputCollectorObserverTest { +public class + SpoutOutputCollectorObserverTest { @Test public void testFlag() { diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestDummyBolt.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestDummyBolt.java index 0fc7df90b97b4..2ad8f2e491236 100644 --- a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestDummyBolt.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestDummyBolt.java @@ -16,13 +16,13 @@ */ package org.apache.flink.storm.util; -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IRichBolt; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Tuple; -import backtype.storm.tuple.Values; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.IRichBolt; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Tuple; +import org.apache.storm.tuple.Values; import java.util.Map; diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestDummySpout.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestDummySpout.java index 7fe8df74c8e09..82506e40d17b5 100644 --- a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestDummySpout.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestDummySpout.java @@ -16,13 +16,13 @@ */ package org.apache.flink.storm.util; -import backtype.storm.spout.SpoutOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IRichSpout; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Values; -import backtype.storm.utils.Utils; +import org.apache.storm.spout.SpoutOutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.IRichSpout; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Values; +import org.apache.storm.utils.Utils; import java.util.Map; diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestSink.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestSink.java index c11597c38ed60..1f4da55f402c3 100644 --- a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestSink.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/util/TestSink.java @@ -16,11 +16,11 @@ */ package org.apache.flink.storm.util; -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IRichBolt; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Tuple; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.IRichBolt; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.tuple.Tuple; import java.util.LinkedList; import java.util.List; diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltCollectorTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltCollectorTest.java index e8748d0a28952..9e3165b31e286 100644 --- a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltCollectorTest.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltCollectorTest.java @@ -17,7 +17,7 @@ package org.apache.flink.storm.wrappers; -import backtype.storm.tuple.Values; +import org.apache.storm.tuple.Values; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.storm.util.AbstractTest; diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltWrapperTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltWrapperTest.java index c15b5f6bf6f52..694be9d547dc0 100644 --- a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltWrapperTest.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltWrapperTest.java @@ -17,14 +17,14 @@ package org.apache.flink.storm.wrappers; -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IRichBolt; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.MessageId; -import backtype.storm.tuple.Values; -import backtype.storm.utils.Utils; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.IRichBolt; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.MessageId; +import org.apache.storm.tuple.Values; +import org.apache.storm.utils.Utils; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.TaskInfo; @@ -337,7 +337,7 @@ public void prepare(Map stormConf, TopologyContext context, OutputCollector coll int counter = 0; @Override - public void execute(backtype.storm.tuple.Tuple input) { + public void execute(org.apache.storm.tuple.Tuple input) { if (++counter % 2 == 1) { this.collector.emit("stream1", new Values(input.getInteger(0))); } else { diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/FlinkTopologyContextTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/FlinkTopologyContextTest.java index de9be2ad159c5..9a23b0f91e0fe 100644 --- a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/FlinkTopologyContextTest.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/FlinkTopologyContextTest.java @@ -17,13 +17,13 @@ package org.apache.flink.storm.wrappers; -import backtype.storm.generated.Bolt; -import backtype.storm.generated.SpoutSpec; -import backtype.storm.generated.StateSpoutSpec; -import backtype.storm.generated.StormTopology; -import backtype.storm.metric.api.ICombiner; -import backtype.storm.metric.api.IMetric; -import backtype.storm.metric.api.IReducer; +import org.apache.storm.generated.Bolt; +import org.apache.storm.generated.SpoutSpec; +import org.apache.storm.generated.StateSpoutSpec; +import org.apache.storm.generated.StormTopology; +import org.apache.storm.metric.api.ICombiner; +import org.apache.storm.metric.api.IMetric; +import org.apache.storm.metric.api.IReducer; import org.apache.flink.storm.util.AbstractTest; import org.junit.Test; diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SetupOutputFieldsDeclarerTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SetupOutputFieldsDeclarerTest.java index 481cb5ca98d4d..94a88fec84608 100644 --- a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SetupOutputFieldsDeclarerTest.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SetupOutputFieldsDeclarerTest.java @@ -17,8 +17,8 @@ package org.apache.flink.storm.wrappers; -import backtype.storm.tuple.Fields; -import backtype.storm.utils.Utils; +import org.apache.storm.tuple.Fields; +import org.apache.storm.utils.Utils; import org.apache.flink.storm.util.AbstractTest; import org.junit.Assert; import org.junit.Test; diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SpoutCollectorTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SpoutCollectorTest.java index fac25827f5b07..eb91c63d80dbe 100644 --- a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SpoutCollectorTest.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SpoutCollectorTest.java @@ -17,7 +17,7 @@ package org.apache.flink.storm.wrappers; -import backtype.storm.tuple.Values; +import org.apache.storm.tuple.Values; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.storm.util.AbstractTest; diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SpoutWrapperTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SpoutWrapperTest.java index e50ff5aaa6b71..1fc99324b4655 100644 --- a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SpoutWrapperTest.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SpoutWrapperTest.java @@ -17,10 +17,10 @@ package org.apache.flink.storm.wrappers; -import backtype.storm.spout.SpoutOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IRichSpout; -import backtype.storm.tuple.Fields; +import org.apache.storm.spout.SpoutOutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.IRichSpout; +import org.apache.storm.tuple.Fields; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.configuration.Configuration; diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/StormTupleTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/StormTupleTest.java index eba611e7b5469..7ea4b76137ae5 100644 --- a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/StormTupleTest.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/StormTupleTest.java @@ -17,10 +17,10 @@ package org.apache.flink.storm.wrappers; -import backtype.storm.generated.GlobalStreamId; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.MessageId; -import backtype.storm.tuple.Values; +import org.apache.storm.generated.GlobalStreamId; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.MessageId; +import org.apache.storm.tuple.Values; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple5; diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupHelperTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupHelperTest.java index 000fe848e8863..6a4ac9ac3e63c 100644 --- a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupHelperTest.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupHelperTest.java @@ -17,42 +17,34 @@ package org.apache.flink.storm.wrappers; -import backtype.storm.Config; -import backtype.storm.LocalCluster; -import backtype.storm.generated.ComponentCommon; -import backtype.storm.generated.StormTopology; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IComponent; -import backtype.storm.topology.IRichBolt; -import backtype.storm.topology.IRichSpout; -import backtype.storm.topology.TopologyBuilder; -import backtype.storm.tuple.Fields; -import backtype.storm.utils.Utils; - +import org.apache.storm.Config; +import org.apache.storm.LocalCluster; +import org.apache.storm.generated.ComponentCommon; +import org.apache.storm.generated.StormTopology; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.IComponent; +import org.apache.storm.topology.IRichBolt; +import org.apache.storm.topology.IRichSpout; +import org.apache.storm.topology.TopologyBuilder; +import org.apache.storm.tuple.Fields; +import org.apache.storm.utils.Utils; import org.apache.flink.storm.api.FlinkTopology; import org.apache.flink.storm.util.AbstractTest; import org.apache.flink.storm.util.TestDummyBolt; import org.apache.flink.storm.util.TestDummySpout; import org.apache.flink.storm.util.TestSink; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; - import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; - import org.powermock.api.mockito.PowerMockito; import org.powermock.core.classloader.annotations.PowerMockIgnore; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Set; +import java.util.*; import static java.util.Collections.singleton; - import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; From 000a5a6dce3f0dd7f2ae157c330d2be9f311d01a Mon Sep 17 00:00:00 2001 From: yuzhongliu Date: Mon, 5 Dec 2016 20:52:20 +0800 Subject: [PATCH 002/299] #FLINK-4450 squash all commits into one commit --- flink-contrib/flink-storm-examples/pom.xml | 56 ++++++++++--------- .../storm/exclamation/ExclamationLocal.java | 10 ++-- .../exclamation/ExclamationTopology.java | 4 +- .../exclamation/ExclamationWithBolt.java | 4 +- .../exclamation/ExclamationWithSpout.java | 4 +- .../operators/ExclamationBolt.java | 14 ++--- .../flink/storm/join/SingleJoinExample.java | 14 ++--- .../flink/storm/print/PrintSampleStream.java | 10 ++-- .../storm/split/operators/RandomSpout.java | 14 ++--- .../split/operators/VerifyAndEnrichBolt.java | 16 +++--- .../flink/storm/util/AbstractBoltSink.java | 10 ++-- .../flink/storm/util/AbstractLineSpout.java | 10 ++-- .../apache/flink/storm/util/BoltFileSink.java | 2 +- .../flink/storm/util/BoltPrintSink.java | 2 +- .../apache/flink/storm/util/FileSpout.java | 6 +- .../flink/storm/util/FiniteFileSpout.java | 6 +- .../flink/storm/util/InMemorySpout.java | 2 +- .../flink/storm/util/OutputFormatter.java | 2 +- .../storm/util/SimpleOutputFormatter.java | 2 +- .../storm/util/TupleOutputFormatter.java | 2 +- .../wordcount/BoltTokenizerWordCount.java | 2 +- .../wordcount/BoltTokenizerWordCountPojo.java | 2 +- .../BoltTokenizerWordCountWithNames.java | 4 +- .../storm/wordcount/SpoutSourceWordCount.java | 4 +- .../flink/storm/wordcount/WordCountLocal.java | 8 +-- .../storm/wordcount/WordCountLocalByName.java | 8 +-- .../wordcount/WordCountRemoteByClient.java | 16 +++--- .../wordcount/WordCountRemoteBySubmitter.java | 8 +-- .../storm/wordcount/WordCountTopology.java | 6 +- .../wordcount/operators/BoltCounter.java | 14 ++--- .../operators/BoltCounterByName.java | 14 ++--- .../wordcount/operators/BoltTokenizer.java | 14 ++--- .../operators/BoltTokenizerByName.java | 14 ++--- .../operators/WordCountFileSpout.java | 4 +- .../operators/WordCountInMemorySpout.java | 4 +- .../apache/flink/storm/split/SplitBolt.java | 16 +++--- .../flink/storm/split/SplitBoltTopology.java | 2 +- .../flink/storm/split/SplitSpoutTopology.java | 2 +- .../storm/split/SplitStreamBoltLocal.java | 4 +- .../storm/split/SplitStreamSpoutLocal.java | 4 +- .../tests/StormFieldsGroupingITCase.java | 8 ++- .../storm/tests/StormMetaDataITCase.java | 4 +- .../flink/storm/tests/StormUnionITCase.java | 4 +- .../tests/operators/FiniteRandomSpout.java | 19 +++---- .../storm/tests/operators/MergerBolt.java | 14 ++--- .../storm/tests/operators/MetaDataSpout.java | 14 ++--- .../storm/tests/operators/TaskIdBolt.java | 16 +++--- .../tests/operators/VerifyMetaDataBolt.java | 18 +++--- flink-contrib/pom.xml | 4 +- pom.xml | 32 +++++------ 50 files changed, 239 insertions(+), 234 deletions(-) diff --git a/flink-contrib/flink-storm-examples/pom.xml b/flink-contrib/flink-storm-examples/pom.xml index 8e92f8389f8b5..83c61676b87aa 100644 --- a/flink-contrib/flink-storm-examples/pom.xml +++ b/flink-contrib/flink-storm-examples/pom.xml @@ -71,7 +71,7 @@ under the License. org.apache.storm storm-starter - 0.9.4 + 1.0.0 @@ -138,7 +138,7 @@ under the License. org.apache.storm storm-core - 0.9.4 + 1.0.0 jar false ${project.build.directory}/classes @@ -197,15 +197,16 @@ under the License. - backtype/storm/topology/*.class - backtype/storm/spout/*.class - backtype/storm/task/*.class - backtype/storm/tuple/*.class - backtype/storm/generated/*.class - backtype/storm/metric/**/*.class - org/apache/thrift7/**/*.class + org/apache/storm/topology/*.class + org/apache/storm/spout/*.class + org/apache/storm/task/*.class + org/apache/storm/tuple/*.class + org/apache/storm/generated/*.class + org/apache/storm/metric/**/*.class + org/apache/storm/thrift/**/*.class org/json/simple/**/*.class + org/apache/storm/shade/**/*.class org/apache/flink/storm/api/*.class org/apache/flink/storm/util/*.class @@ -243,15 +244,16 @@ under the License. - backtype/storm/topology/*.class - backtype/storm/spout/*.class - backtype/storm/task/*.class - backtype/storm/tuple/*.class - backtype/storm/generated/*.class - backtype/storm/metric/**/*.class - org/apache/thrift7/**/*.class + org/apache/storm/topology/*.class + org/apache/storm/spout/*.class + org/apache/storm/task/*.class + org/apache/storm/tuple/*.class + org/apache/storm/generated/*.class + org/apache/storm/metric/**/*.class + org/apache/storm/thrift/**/*.class org/json/simple/**/*.class + org/apache/storm/shade/**/*.class org/apache/flink/storm/api/*.class org/apache/flink/storm/util/*.class @@ -322,20 +324,22 @@ under the License. org.apache.storm:storm-core defaults.yaml - backtype/storm/*.class - backtype/storm/topology/*.class - backtype/storm/spout/*.class - backtype/storm/task/*.class - backtype/storm/tuple/*.class - backtype/storm/generated/*.class - backtype/storm/metric/**/*.class - backtype/storm/utils/*.class - backtype/storm/serialization/*.class + org/apache/storm/*.class + org/apache/storm/topology/*.class + org/apache/storm/spout/*.class + org/apache/storm/task/*.class + org/apache/storm/tuple/*.class + org/apache/storm/generated/*.class + org/apache/storm/metric/**/*.class + org/apache/storm/utils/*.class + org/apache/storm/serialization/*.class org/apache/storm/curator/**/*.class - org/apache/thrift7/**/*.class + org/apache/storm/grouping/**/*.class + org/apache/storm/thrift/**/*.class org/json/simple/**/*.class org/yaml/snakeyaml/**/*.class + org/apache/storm/shade/**/*.class diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/exclamation/ExclamationLocal.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/exclamation/ExclamationLocal.java index 3f2d8064cf02b..c37ae65f7a57c 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/exclamation/ExclamationLocal.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/exclamation/ExclamationLocal.java @@ -17,19 +17,19 @@ package org.apache.flink.storm.exclamation; -import backtype.storm.Config; -import backtype.storm.topology.TopologyBuilder; +import org.apache.storm.Config; +import org.apache.storm.topology.TopologyBuilder; import org.apache.flink.storm.api.FlinkLocalCluster; import org.apache.flink.storm.api.FlinkTopology; import org.apache.flink.storm.exclamation.operators.ExclamationBolt; /** * Implements the "Exclamation" program that attaches five exclamation mark to every line of a text files in a streaming - * fashion. The program is constructed as a regular {@link backtype.storm.generated.StormTopology} and submitted to - * Flink for execution in the same way as to a Storm {@link backtype.storm.LocalCluster}. + * fashion. The program is constructed as a regular {@link org.apache.storm.generated.StormTopology} and submitted to + * Flink for execution in the same way as to a Storm {@link org.apache.storm.LocalCluster}. *

* This example shows how to run program directly within Java, thus it cannot be used to submit a - * {@link backtype.storm.generated.StormTopology} via Flink command line clients (ie, bin/flink). + * {@link org.apache.storm.generated.StormTopology} via Flink command line clients (ie, bin/flink). *

* The input is a plain text file with lines separated by newline characters. *

diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/exclamation/ExclamationTopology.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/exclamation/ExclamationTopology.java index 43f526b33f28b..0144acbfa62bf 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/exclamation/ExclamationTopology.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/exclamation/ExclamationTopology.java @@ -17,7 +17,7 @@ package org.apache.flink.storm.exclamation; -import backtype.storm.topology.TopologyBuilder; +import org.apache.storm.topology.TopologyBuilder; import org.apache.flink.examples.java.wordcount.util.WordCountData; import org.apache.flink.storm.exclamation.operators.ExclamationBolt; import org.apache.flink.storm.util.BoltFileSink; @@ -29,7 +29,7 @@ /** * Implements the "Exclamation" program that attaches two exclamation marks to every line of a text files in a streaming - * fashion. The program is constructed as a regular {@link backtype.storm.generated.StormTopology}. + * fashion. The program is constructed as a regular {@link org.apache.storm.generated.StormTopology}. *

* The input is a plain text file with lines separated by newline characters. *

diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/exclamation/ExclamationWithBolt.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/exclamation/ExclamationWithBolt.java index b47c0fa19c51b..5a791198382ac 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/exclamation/ExclamationWithBolt.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/exclamation/ExclamationWithBolt.java @@ -18,7 +18,7 @@ package org.apache.flink.storm.exclamation; -import backtype.storm.utils.Utils; +import org.apache.storm.utils.Utils; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.examples.java.wordcount.util.WordCountData; @@ -30,7 +30,7 @@ /** * Implements the "Exclamation" program that attaches 3+x exclamation marks to every line of a text files in a streaming - * fashion. The program is constructed as a regular {@link backtype.storm.generated.StormTopology}. + * fashion. The program is constructed as a regular {@link org.apache.storm.generated.StormTopology}. *

* The input is a plain text file with lines separated by newline characters. *

diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/exclamation/ExclamationWithSpout.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/exclamation/ExclamationWithSpout.java index 380d9daa2d85f..237f1d4e4c436 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/exclamation/ExclamationWithSpout.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/exclamation/ExclamationWithSpout.java @@ -18,7 +18,7 @@ package org.apache.flink.storm.exclamation; -import backtype.storm.utils.Utils; +import org.apache.storm.utils.Utils; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.examples.java.wordcount.util.WordCountData; @@ -31,7 +31,7 @@ /** * Implements the "Exclamation" program that attaches six exclamation marks to every line of a text files in a streaming - * fashion. The program is constructed as a regular {@link backtype.storm.generated.StormTopology}. + * fashion. The program is constructed as a regular {@link org.apache.storm.generated.StormTopology}. *

* The input is a plain text file with lines separated by newline characters. *

diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/exclamation/operators/ExclamationBolt.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/exclamation/operators/ExclamationBolt.java index 9bc00d27ed6a6..77a91d2e2a3c9 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/exclamation/operators/ExclamationBolt.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/exclamation/operators/ExclamationBolt.java @@ -18,13 +18,13 @@ package org.apache.flink.storm.exclamation.operators; -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IRichBolt; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Tuple; -import backtype.storm.tuple.Values; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.IRichBolt; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Tuple; +import org.apache.storm.tuple.Values; import java.util.Map; diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/join/SingleJoinExample.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/join/SingleJoinExample.java index 3ccd885c283d4..41ea4cbdbfa7f 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/join/SingleJoinExample.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/join/SingleJoinExample.java @@ -17,11 +17,11 @@ */ package org.apache.flink.storm.join; -import backtype.storm.Config; -import backtype.storm.testing.FeederSpout; -import backtype.storm.topology.TopologyBuilder; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Values; +import org.apache.storm.Config; +import org.apache.storm.testing.FeederSpout; +import org.apache.storm.topology.TopologyBuilder; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Values; import org.apache.flink.storm.api.FlinkLocalCluster; import org.apache.flink.storm.api.FlinkTopology; @@ -29,8 +29,8 @@ import org.apache.flink.storm.util.NullTerminatingSpout; import org.apache.flink.storm.util.TupleOutputFormatter; -import storm.starter.bolt.PrinterBolt; -import storm.starter.bolt.SingleJoinBolt; +import org.apache.storm.starter.bolt.PrinterBolt; +import org.apache.storm.starter.bolt.SingleJoinBolt; public class SingleJoinExample { diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/print/PrintSampleStream.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/print/PrintSampleStream.java index de652cf9c2883..da2e641f9c186 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/print/PrintSampleStream.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/print/PrintSampleStream.java @@ -18,13 +18,13 @@ package org.apache.flink.storm.print; -import backtype.storm.Config; -import backtype.storm.topology.TopologyBuilder; -import backtype.storm.utils.Utils; +import org.apache.storm.Config; +import org.apache.storm.topology.TopologyBuilder; +import org.apache.storm.utils.Utils; import org.apache.flink.storm.api.FlinkLocalCluster; import org.apache.flink.storm.api.FlinkTopology; -import storm.starter.bolt.PrinterBolt; -import storm.starter.spout.TwitterSampleSpout; +import org.apache.storm.starter.bolt.PrinterBolt; +import org.apache.storm.starter.spout.TwitterSampleSpout; import java.util.Arrays; diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/operators/RandomSpout.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/operators/RandomSpout.java index d3153957beba5..cfa427fc58ce5 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/operators/RandomSpout.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/operators/RandomSpout.java @@ -17,16 +17,16 @@ */ package org.apache.flink.storm.split.operators; +import org.apache.storm.spout.SpoutOutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.topology.base.BaseRichSpout; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Values; + import java.util.Map; import java.util.Random; -import backtype.storm.spout.SpoutOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.topology.base.BaseRichSpout; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Values; - public class RandomSpout extends BaseRichSpout { private static final long serialVersionUID = -3978554318742509334L; diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/operators/VerifyAndEnrichBolt.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/operators/VerifyAndEnrichBolt.java index 434d091247795..312ca78e9e337 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/operators/VerifyAndEnrichBolt.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/split/operators/VerifyAndEnrichBolt.java @@ -17,15 +17,15 @@ */ package org.apache.flink.storm.split.operators; -import java.util.Map; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.topology.base.BaseRichBolt; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Tuple; +import org.apache.storm.tuple.Values; -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.topology.base.BaseRichBolt; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Tuple; -import backtype.storm.tuple.Values; +import java.util.Map; public class VerifyAndEnrichBolt extends BaseRichBolt { private static final long serialVersionUID = -7277395570966328721L; diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/AbstractBoltSink.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/AbstractBoltSink.java index a6c61d43f3e82..2cb346ae7c126 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/AbstractBoltSink.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/AbstractBoltSink.java @@ -17,11 +17,11 @@ package org.apache.flink.storm.util; -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IRichBolt; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Tuple; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.IRichBolt; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.tuple.Tuple; import java.util.Map; diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/AbstractLineSpout.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/AbstractLineSpout.java index d19ffbf0606ac..29df23ece4e32 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/AbstractLineSpout.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/AbstractLineSpout.java @@ -17,11 +17,11 @@ package org.apache.flink.storm.util; -import backtype.storm.spout.SpoutOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IRichSpout; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Fields; +import org.apache.storm.spout.SpoutOutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.IRichSpout; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.tuple.Fields; import java.util.Map; diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/BoltFileSink.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/BoltFileSink.java index 5cd3f68b86f1c..cbbe1917955bb 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/BoltFileSink.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/BoltFileSink.java @@ -17,7 +17,7 @@ package org.apache.flink.storm.util; -import backtype.storm.task.TopologyContext; +import org.apache.storm.task.TopologyContext; import java.io.BufferedWriter; import java.io.FileWriter; diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/BoltPrintSink.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/BoltPrintSink.java index 044246b114c90..a80417b01ab2c 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/BoltPrintSink.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/BoltPrintSink.java @@ -17,7 +17,7 @@ package org.apache.flink.storm.util; -import backtype.storm.task.TopologyContext; +import org.apache.storm.task.TopologyContext; import java.util.Map; diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FileSpout.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FileSpout.java index 1126a2a0c8d6b..0a295e771790c 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FileSpout.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FileSpout.java @@ -17,9 +17,9 @@ package org.apache.flink.storm.util; -import backtype.storm.spout.SpoutOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.tuple.Values; +import org.apache.storm.spout.SpoutOutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.tuple.Values; import java.io.BufferedReader; import java.io.FileNotFoundException; diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FiniteFileSpout.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FiniteFileSpout.java index 4b41f8ad16391..48349c2803a7a 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FiniteFileSpout.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/FiniteFileSpout.java @@ -16,9 +16,9 @@ */ package org.apache.flink.storm.util; -import backtype.storm.spout.SpoutOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.tuple.Values; +import org.apache.storm.spout.SpoutOutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.tuple.Values; import java.io.IOException; import java.util.Map; diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/InMemorySpout.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/InMemorySpout.java index 5e4c7ba8f9155..de1ca20ebf128 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/InMemorySpout.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/InMemorySpout.java @@ -17,7 +17,7 @@ package org.apache.flink.storm.util; -import backtype.storm.tuple.Values; +import org.apache.storm.tuple.Values; /** * Implements a Spout that reads data stored in memory. diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/OutputFormatter.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/OutputFormatter.java index e696f9bc2dac5..fe28afc87262c 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/OutputFormatter.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/OutputFormatter.java @@ -18,7 +18,7 @@ package org.apache.flink.storm.util; -import backtype.storm.tuple.Tuple; +import org.apache.storm.tuple.Tuple; import java.io.Serializable; diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/SimpleOutputFormatter.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/SimpleOutputFormatter.java index cef008105b4f8..323fb53884229 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/SimpleOutputFormatter.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/SimpleOutputFormatter.java @@ -18,7 +18,7 @@ package org.apache.flink.storm.util; -import backtype.storm.tuple.Tuple; +import org.apache.storm.tuple.Tuple; public class SimpleOutputFormatter implements OutputFormatter { private static final long serialVersionUID = 6349573860144270338L; diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/TupleOutputFormatter.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/TupleOutputFormatter.java index 5d7ba53eaa535..11d23cddaccdb 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/TupleOutputFormatter.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/util/TupleOutputFormatter.java @@ -18,7 +18,7 @@ package org.apache.flink.storm.util; -import backtype.storm.tuple.Tuple; +import org.apache.storm.tuple.Tuple; public class TupleOutputFormatter implements OutputFormatter { private static final long serialVersionUID = -599665757723851761L; diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCount.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCount.java index cccf4c0533eed..4620d9d0cf167 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCount.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCount.java @@ -17,7 +17,7 @@ package org.apache.flink.storm.wordcount; -import backtype.storm.topology.IRichBolt; +import org.apache.storm.topology.IRichBolt; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.examples.java.wordcount.util.WordCountData; diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountPojo.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountPojo.java index 9b0d4ee6fc655..eefbf78481ac0 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountPojo.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountPojo.java @@ -17,7 +17,7 @@ package org.apache.flink.storm.wordcount; -import backtype.storm.topology.IRichBolt; +import org.apache.storm.topology.IRichBolt; import org.apache.flink.api.java.io.CsvInputFormat; import org.apache.flink.api.java.io.PojoCsvInputFormat; import org.apache.flink.api.java.tuple.Tuple2; diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountWithNames.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountWithNames.java index 50d4518f9310f..98f7f96324b1e 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountWithNames.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/BoltTokenizerWordCountWithNames.java @@ -17,8 +17,8 @@ package org.apache.flink.storm.wordcount; -import backtype.storm.topology.IRichBolt; -import backtype.storm.tuple.Fields; +import org.apache.storm.topology.IRichBolt; +import org.apache.storm.tuple.Fields; import org.apache.flink.api.java.io.CsvInputFormat; import org.apache.flink.api.java.io.TupleCsvInputFormat; import org.apache.flink.api.java.tuple.Tuple; diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/SpoutSourceWordCount.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/SpoutSourceWordCount.java index 281780ea9e263..683a3b5480bfc 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/SpoutSourceWordCount.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/SpoutSourceWordCount.java @@ -17,8 +17,8 @@ package org.apache.flink.storm.wordcount; -import backtype.storm.topology.IRichSpout; -import backtype.storm.utils.Utils; +import org.apache.storm.topology.IRichSpout; +import org.apache.storm.utils.Utils; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.java.tuple.Tuple2; diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocal.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocal.java index 2fa79ac790bc2..ee880badf9e56 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocal.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocal.java @@ -17,10 +17,10 @@ package org.apache.flink.storm.wordcount; -import backtype.storm.Config; -import backtype.storm.LocalCluster; -import backtype.storm.generated.StormTopology; -import backtype.storm.topology.TopologyBuilder; +import org.apache.storm.Config; +import org.apache.storm.LocalCluster; +import org.apache.storm.generated.StormTopology; +import org.apache.storm.topology.TopologyBuilder; import org.apache.flink.examples.java.wordcount.util.WordCountData; import org.apache.flink.storm.api.FlinkLocalCluster; diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocalByName.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocalByName.java index 046bc0422adc1..ab423cf29e663 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocalByName.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountLocalByName.java @@ -17,10 +17,10 @@ package org.apache.flink.storm.wordcount; -import backtype.storm.Config; -import backtype.storm.LocalCluster; -import backtype.storm.generated.StormTopology; -import backtype.storm.topology.TopologyBuilder; +import org.apache.storm.Config; +import org.apache.storm.LocalCluster; +import org.apache.storm.generated.StormTopology; +import org.apache.storm.topology.TopologyBuilder; import org.apache.flink.examples.java.wordcount.util.WordCountData; import org.apache.flink.storm.api.FlinkLocalCluster; diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteByClient.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteByClient.java index 6b718b9ae564e..5c99f933c71ff 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteByClient.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteByClient.java @@ -17,14 +17,14 @@ package org.apache.flink.storm.wordcount; -import backtype.storm.Config; -import backtype.storm.generated.AlreadyAliveException; -import backtype.storm.generated.InvalidTopologyException; -import backtype.storm.generated.NotAliveException; -import backtype.storm.generated.StormTopology; -import backtype.storm.topology.TopologyBuilder; -import backtype.storm.utils.NimbusClient; -import backtype.storm.utils.Utils; +import org.apache.storm.Config; +import org.apache.storm.generated.AlreadyAliveException; +import org.apache.storm.generated.InvalidTopologyException; +import org.apache.storm.generated.NotAliveException; +import org.apache.storm.generated.StormTopology; +import org.apache.storm.topology.TopologyBuilder; +import org.apache.storm.utils.NimbusClient; +import org.apache.storm.utils.Utils; import org.apache.flink.examples.java.wordcount.util.WordCountData; import org.apache.flink.storm.api.FlinkClient; diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteBySubmitter.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteBySubmitter.java index eb2713dcb13c3..08ba52aeccd26 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteBySubmitter.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountRemoteBySubmitter.java @@ -17,11 +17,11 @@ package org.apache.flink.storm.wordcount; -import backtype.storm.Config; -import backtype.storm.StormSubmitter; -import backtype.storm.generated.StormTopology; +import org.apache.storm.Config; +import org.apache.storm.StormSubmitter; +import org.apache.storm.generated.StormTopology; -import backtype.storm.topology.TopologyBuilder; +import org.apache.storm.topology.TopologyBuilder; import org.apache.flink.examples.java.wordcount.util.WordCountData; import org.apache.flink.storm.api.FlinkClient; import org.apache.flink.storm.api.FlinkSubmitter; diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountTopology.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountTopology.java index e4117f4773cd4..8f855b5ce8610 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountTopology.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/WordCountTopology.java @@ -17,9 +17,9 @@ package org.apache.flink.storm.wordcount; -import backtype.storm.generated.StormTopology; -import backtype.storm.topology.TopologyBuilder; -import backtype.storm.tuple.Fields; +import org.apache.storm.generated.StormTopology; +import org.apache.storm.topology.TopologyBuilder; +import org.apache.storm.tuple.Fields; import org.apache.flink.examples.java.wordcount.util.WordCountData; import org.apache.flink.storm.util.BoltFileSink; diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltCounter.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltCounter.java index d21a58435eadf..4a00869c9f4e2 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltCounter.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltCounter.java @@ -17,13 +17,13 @@ package org.apache.flink.storm.wordcount.operators; -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IRichBolt; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Tuple; -import backtype.storm.tuple.Values; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.IRichBolt; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Tuple; +import org.apache.storm.tuple.Values; import java.util.HashMap; import java.util.Map; diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltCounterByName.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltCounterByName.java index d5c05d7937346..e3e0d58f9db19 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltCounterByName.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltCounterByName.java @@ -17,13 +17,13 @@ package org.apache.flink.storm.wordcount.operators; -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IRichBolt; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Tuple; -import backtype.storm.tuple.Values; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.IRichBolt; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Tuple; +import org.apache.storm.tuple.Values; import java.util.HashMap; import java.util.Map; diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltTokenizer.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltTokenizer.java index 74d6a991d9da0..cedd90a5a44db 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltTokenizer.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltTokenizer.java @@ -17,13 +17,13 @@ package org.apache.flink.storm.wordcount.operators; -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IRichBolt; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Tuple; -import backtype.storm.tuple.Values; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.IRichBolt; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Tuple; +import org.apache.storm.tuple.Values; import java.util.Map; diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltTokenizerByName.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltTokenizerByName.java index 3c56b3608a439..258d4129d34a3 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltTokenizerByName.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/BoltTokenizerByName.java @@ -17,13 +17,13 @@ package org.apache.flink.storm.wordcount.operators; -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.IRichBolt; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Tuple; -import backtype.storm.tuple.Values; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.IRichBolt; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Tuple; +import org.apache.storm.tuple.Values; import java.util.Map; diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountFileSpout.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountFileSpout.java index 76a198f80b474..1298422ec6d45 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountFileSpout.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountFileSpout.java @@ -19,8 +19,8 @@ import org.apache.flink.storm.util.FileSpout; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Fields; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.tuple.Fields; /** * Implements a Spout that reads data from a given local file. diff --git a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountInMemorySpout.java b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountInMemorySpout.java index c06c268246b94..7bf40c2fcadf1 100644 --- a/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountInMemorySpout.java +++ b/flink-contrib/flink-storm-examples/src/main/java/org/apache/flink/storm/wordcount/operators/WordCountInMemorySpout.java @@ -17,8 +17,8 @@ package org.apache.flink.storm.wordcount.operators; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.tuple.Fields; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.tuple.Fields; import org.apache.flink.examples.java.wordcount.util.WordCountData; import org.apache.flink.storm.util.FiniteInMemorySpout; diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitBolt.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitBolt.java index c7b9c1d400008..fac214c3ce19b 100644 --- a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitBolt.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitBolt.java @@ -17,15 +17,15 @@ */ package org.apache.flink.storm.split; -import java.util.Map; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.topology.base.BaseRichBolt; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Tuple; +import org.apache.storm.tuple.Values; -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.topology.base.BaseRichBolt; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Tuple; -import backtype.storm.tuple.Values; +import java.util.Map; public class SplitBolt extends BaseRichBolt { private static final long serialVersionUID = -6627606934204267173L; diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitBoltTopology.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitBoltTopology.java index 52fbc2ac0fe86..04cfeedac4ede 100644 --- a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitBoltTopology.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitBoltTopology.java @@ -17,7 +17,7 @@ */ package org.apache.flink.storm.split; -import backtype.storm.topology.TopologyBuilder; +import org.apache.storm.topology.TopologyBuilder; import org.apache.flink.storm.split.operators.RandomSpout; import org.apache.flink.storm.split.operators.VerifyAndEnrichBolt; import org.apache.flink.storm.util.BoltFileSink; diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitSpoutTopology.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitSpoutTopology.java index 2527616283830..8671d2e91ccb9 100644 --- a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitSpoutTopology.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitSpoutTopology.java @@ -17,7 +17,7 @@ */ package org.apache.flink.storm.split; -import backtype.storm.topology.TopologyBuilder; +import org.apache.storm.topology.TopologyBuilder; import org.apache.flink.storm.split.operators.RandomSpout; import org.apache.flink.storm.split.operators.VerifyAndEnrichBolt; import org.apache.flink.storm.util.BoltFileSink; diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitStreamBoltLocal.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitStreamBoltLocal.java index ad334aee2579f..2cde11e953dd9 100644 --- a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitStreamBoltLocal.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitStreamBoltLocal.java @@ -16,8 +16,8 @@ */ package org.apache.flink.storm.split; -import backtype.storm.topology.TopologyBuilder; -import backtype.storm.utils.Utils; +import org.apache.storm.topology.TopologyBuilder; +import org.apache.storm.utils.Utils; import org.apache.flink.storm.api.FlinkLocalCluster; import org.apache.flink.storm.api.FlinkTopology; diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitStreamSpoutLocal.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitStreamSpoutLocal.java index 69b40e8f57ec3..be880d0187588 100644 --- a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitStreamSpoutLocal.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/split/SplitStreamSpoutLocal.java @@ -16,8 +16,8 @@ */ package org.apache.flink.storm.split; -import backtype.storm.topology.TopologyBuilder; -import backtype.storm.utils.Utils; +import org.apache.storm.topology.TopologyBuilder; +import org.apache.storm.utils.Utils; import org.apache.flink.storm.api.FlinkLocalCluster; import org.apache.flink.storm.api.FlinkTopology; diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/StormFieldsGroupingITCase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/StormFieldsGroupingITCase.java index 5df133702ac0a..5297d4a79c933 100644 --- a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/StormFieldsGroupingITCase.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/StormFieldsGroupingITCase.java @@ -17,9 +17,11 @@ */ package org.apache.flink.storm.tests; -import backtype.storm.Config; -import backtype.storm.topology.TopologyBuilder; -import backtype.storm.tuple.Fields; +import org.apache.storm.Config; +import org.apache.storm.topology.TopologyBuilder; +import org.apache.storm.tuple.Fields; + +import org.apache.flink.util.MathUtils; import org.apache.flink.storm.api.FlinkLocalCluster; import org.apache.flink.storm.api.FlinkTopology; import org.apache.flink.storm.tests.operators.FiniteRandomSpout; diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/StormMetaDataITCase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/StormMetaDataITCase.java index ce869dfdceb31..b19e106328d81 100644 --- a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/StormMetaDataITCase.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/StormMetaDataITCase.java @@ -17,8 +17,8 @@ */ package org.apache.flink.storm.tests; -import backtype.storm.topology.TopologyBuilder; -import backtype.storm.utils.Utils; +import org.apache.storm.topology.TopologyBuilder; +import org.apache.storm.utils.Utils; import org.apache.flink.storm.api.FlinkLocalCluster; import org.apache.flink.storm.api.FlinkTopology; diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/StormUnionITCase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/StormUnionITCase.java index 2518f35dd22bc..452fef518daf5 100644 --- a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/StormUnionITCase.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/StormUnionITCase.java @@ -17,8 +17,8 @@ */ package org.apache.flink.storm.tests; -import backtype.storm.Config; -import backtype.storm.topology.TopologyBuilder; +import org.apache.storm.Config; +import org.apache.storm.topology.TopologyBuilder; import org.apache.flink.storm.api.FlinkLocalCluster; import org.apache.flink.storm.api.FlinkTopology; diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/operators/FiniteRandomSpout.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/operators/FiniteRandomSpout.java index 39072ebc1976c..918acd27c50e9 100644 --- a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/operators/FiniteRandomSpout.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/operators/FiniteRandomSpout.java @@ -17,18 +17,17 @@ */ package org.apache.flink.storm.tests.operators; -import java.util.Map; -import java.util.Random; - import org.apache.flink.storm.util.FiniteSpout; +import org.apache.storm.spout.SpoutOutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.topology.base.BaseRichSpout; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Values; +import org.apache.storm.utils.Utils; -import backtype.storm.spout.SpoutOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.topology.base.BaseRichSpout; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Values; -import backtype.storm.utils.Utils; +import java.util.Map; +import java.util.Random; public class FiniteRandomSpout extends BaseRichSpout implements FiniteSpout { private static final long serialVersionUID = 6592885571932363239L; diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/operators/MergerBolt.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/operators/MergerBolt.java index 2c353bfc8f124..ababed044eeed 100644 --- a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/operators/MergerBolt.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/operators/MergerBolt.java @@ -17,14 +17,14 @@ */ package org.apache.flink.storm.tests.operators; -import java.util.Map; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.topology.base.BaseRichBolt; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Tuple; -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.topology.base.BaseRichBolt; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Tuple; +import java.util.Map; public class MergerBolt extends BaseRichBolt { private static final long serialVersionUID = -7966475984592762720L; diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/operators/MetaDataSpout.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/operators/MetaDataSpout.java index 29379090c4d29..267b4451031e6 100644 --- a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/operators/MetaDataSpout.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/operators/MetaDataSpout.java @@ -17,14 +17,14 @@ */ package org.apache.flink.storm.tests.operators; -import java.util.Map; +import org.apache.storm.spout.SpoutOutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.topology.base.BaseRichSpout; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Values; -import backtype.storm.spout.SpoutOutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.topology.base.BaseRichSpout; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Values; +import java.util.Map; public class MetaDataSpout extends BaseRichSpout { private static final long serialVersionUID = 5305870218033256376L; diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/operators/TaskIdBolt.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/operators/TaskIdBolt.java index b69dde7276493..f3afafa6755ba 100644 --- a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/operators/TaskIdBolt.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/operators/TaskIdBolt.java @@ -17,15 +17,15 @@ */ package org.apache.flink.storm.tests.operators; -import java.util.Map; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.topology.base.BaseRichBolt; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Tuple; +import org.apache.storm.tuple.Values; -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.topology.base.BaseRichBolt; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.Tuple; -import backtype.storm.tuple.Values; +import java.util.Map; /** * Bolt to prepend all incoming tuple values with the task id. diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/operators/VerifyMetaDataBolt.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/operators/VerifyMetaDataBolt.java index a02f1f9e3e831..0823a8b1724ae 100644 --- a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/operators/VerifyMetaDataBolt.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/tests/operators/VerifyMetaDataBolt.java @@ -17,16 +17,16 @@ */ package org.apache.flink.storm.tests.operators; -import java.util.Map; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.topology.base.BaseRichBolt; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.MessageId; +import org.apache.storm.tuple.Tuple; +import org.apache.storm.tuple.Values; -import backtype.storm.task.OutputCollector; -import backtype.storm.task.TopologyContext; -import backtype.storm.topology.OutputFieldsDeclarer; -import backtype.storm.topology.base.BaseRichBolt; -import backtype.storm.tuple.Fields; -import backtype.storm.tuple.MessageId; -import backtype.storm.tuple.Tuple; -import backtype.storm.tuple.Values; +import java.util.Map; public class VerifyMetaDataBolt extends BaseRichBolt { private static final long serialVersionUID = 1353222852073800478L; diff --git a/flink-contrib/pom.xml b/flink-contrib/pom.xml index be344462d0866..89d5f8cf2a1b6 100644 --- a/flink-contrib/pom.xml +++ b/flink-contrib/pom.xml @@ -40,9 +40,9 @@ under the License. flink-storm flink-storm-examples flink-streaming-contrib - flink-tweet-inputformat + diff --git a/pom.xml b/pom.xml index 02e868e83eac2..c05b1b21c529a 100644 --- a/pom.xml +++ b/pom.xml @@ -63,15 +63,15 @@ under the License. flink-optimizer flink-streaming-java flink-streaming-scala - flink-streaming-connectors + flink-streaming-connectors flink-batch-connectors flink-examples flink-clients - flink-tests + flink-tests flink-test-utils-parent flink-libraries flink-scala-shell - flink-quickstart + flink-quickstart--> flink-contrib flink-dist flink-mesos @@ -122,7 +122,7 @@ under the License. com.google.code.findbugs jsr305 - + org.apache.commons commons-lang3 @@ -150,7 +150,7 @@ under the License. junit junit - ${junit.version} + ${junit.versionStormFieldsGroupingITCase} jar test @@ -190,9 +190,9 @@ under the License. - org.apache.avro avro 1.7.6 - + org.apache.avro avro-ipc @@ -287,7 +287,7 @@ under the License. joda-convert 1.7 - + com.esotericsoftware.kryo @@ -410,7 +410,7 @@ under the License. - + scala-2.11 @@ -469,7 +469,7 @@ under the License. flink-yarn-tests - + vendor-repos @@ -657,7 +657,7 @@ under the License. - + release @@ -1076,7 +1076,7 @@ under the License. - + @@ -1087,7 +1087,7 @@ under the License. maven-shade-plugin 2.4.1 - + org.eclipse.m2e From 7d9eb73657248ff8fd93f796f39f843b331b57d8 Mon Sep 17 00:00:00 2001 From: yuzhongliu Date: Mon, 5 Dec 2016 20:53:28 +0800 Subject: [PATCH 003/299] #FLINK-4450 squash all commits into one commit --- flink-contrib/pom.xml | 4 ++-- pom.xml | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-contrib/pom.xml b/flink-contrib/pom.xml index 89d5f8cf2a1b6..be344462d0866 100644 --- a/flink-contrib/pom.xml +++ b/flink-contrib/pom.xml @@ -40,9 +40,9 @@ under the License. flink-storm flink-storm-examples flink-streaming-contrib - + flink-statebackend-rocksdb diff --git a/pom.xml b/pom.xml index c05b1b21c529a..f27eb8bab2e33 100644 --- a/pom.xml +++ b/pom.xml @@ -67,11 +67,11 @@ under the License. flink-batch-connectors flink-examples flink-clients - flink-tests + flink-tests flink-test-utils-parent flink-libraries flink-scala-shell - flink-quickstart--> + flink-quickstart flink-contrib flink-dist flink-mesos From e76a0bc43afa6ed53304bdd874bf57b1d235ceea Mon Sep 17 00:00:00 2001 From: yuzhongliu Date: Tue, 30 Aug 2016 14:12:00 +0800 Subject: [PATCH 004/299] #4450 reverse for uncorrect change submit pom.xml --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index f27eb8bab2e33..d92944fa87f83 100644 --- a/pom.xml +++ b/pom.xml @@ -63,7 +63,7 @@ under the License. flink-optimizer flink-streaming-java flink-streaming-scala - flink-streaming-connectors + flink-streaming-connectors flink-batch-connectors flink-examples flink-clients From e6a77ba2214becf45ae9d7f78bafbd54481ef5e8 Mon Sep 17 00:00:00 2001 From: yuzhongliu Date: Wed, 31 Aug 2016 11:12:14 +0800 Subject: [PATCH 005/299] #4450 exclusion unnecessary dependency in flink-storm --- flink-contrib/flink-storm/pom.xml | 60 +++++++++---------------------- 1 file changed, 16 insertions(+), 44 deletions(-) diff --git a/flink-contrib/flink-storm/pom.xml b/flink-contrib/flink-storm/pom.xml index d5def18305ecf..2b0f84de01dda 100644 --- a/flink-contrib/flink-storm/pom.xml +++ b/flink-contrib/flink-storm/pom.xml @@ -78,64 +78,36 @@ under the License. log4j-over-slf4j - ch.qos.logback - logback-classic + org.apache.logging.log4j + log4j-slf4j-impl - ring - ring-core - - - ring - ring-devel - - - ring - ring-servlet - - - ring - ring-jetty-adapter - - - org.mortbay.jetty - jetty - - - org.mortbay.jetty - jetty-util - - - org.jgrapht - jgrapht-core - - - compojure - compojure + slf4j-log4j12 + org.slf4j - com.esotericsoftware.reflectasm - reflectasm + javax.servlet + servlet-api - com.twitter - chill-java + junit + junit - commons-fileupload - commons-fileupload + org.mockito + mockito-all - javax.servlet - servlet-api + org.mockito + mockito-all - clout - clout + org.apache.curator + curator-test - hiccup - hiccup + com.esotericsoftware + kryo From d615cd65214fab2b2fe810a8966f949fd1a286fb Mon Sep 17 00:00:00 2001 From: yuzhongliu Date: Wed, 31 Aug 2016 11:14:21 +0800 Subject: [PATCH 006/299] #4450 delete LocalCluster storm topology test --- .../wrappers/WrapperSetupHelperTest.java | 29 +++++++++---------- 1 file changed, 14 insertions(+), 15 deletions(-) diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupHelperTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupHelperTest.java index 6a4ac9ac3e63c..ec59062e52da6 100644 --- a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupHelperTest.java +++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupHelperTest.java @@ -17,8 +17,13 @@ package org.apache.flink.storm.wrappers; +import org.apache.flink.storm.api.FlinkTopology; +import org.apache.flink.storm.util.AbstractTest; +import org.apache.flink.storm.util.TestDummyBolt; +import org.apache.flink.storm.util.TestDummySpout; +import org.apache.flink.storm.util.TestSink; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.storm.Config; -import org.apache.storm.LocalCluster; import org.apache.storm.generated.ComponentCommon; import org.apache.storm.generated.StormTopology; import org.apache.storm.task.TopologyContext; @@ -28,12 +33,6 @@ import org.apache.storm.topology.TopologyBuilder; import org.apache.storm.tuple.Fields; import org.apache.storm.utils.Utils; -import org.apache.flink.storm.api.FlinkTopology; -import org.apache.flink.storm.util.AbstractTest; -import org.apache.flink.storm.util.TestDummyBolt; -import org.apache.flink.storm.util.TestDummySpout; -import org.apache.flink.storm.util.TestSink; -import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -178,15 +177,15 @@ public void testCreateTopologyContext() { .shuffleGrouping("bolt2", TestDummyBolt.groupingStreamId) .shuffleGrouping("bolt2", TestDummyBolt.shuffleStreamId); - LocalCluster cluster = new LocalCluster(); - Config c = new Config(); - c.setNumAckers(0); - cluster.submitTopology("test", c, builder.createTopology()); +// LocalCluster cluster = new LocalCluster(); +// Config c = new Config(); +// c.setNumAckers(0); +// cluster.submitTopology("test", c, builder.createTopology()); - while (TestSink.result.size() != 8) { - Utils.sleep(100); - } - cluster.shutdown(); +// while (TestSink.result.size() != 8) { +// Utils.sleep(100); +// } +// cluster.shutdown(); final FlinkTopology flinkBuilder = FlinkTopology.createTopology(builder); StormTopology stormTopology = flinkBuilder.getStormTopology(); From 3e7a8ebbc6235a31b59ec2e1d74d20120f6518fb Mon Sep 17 00:00:00 2001 From: kl0u Date: Wed, 31 Aug 2016 17:32:53 +0200 Subject: [PATCH 007/299] [hotfix] Fixes unstable ContinuousFileMonitoringTest. This closes #2446 --- .../ContinuousFileMonitoringTest.java | 60 +++++++++++-------- 1 file changed, 36 insertions(+), 24 deletions(-) diff --git a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileMonitoringTest.java b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileMonitoringTest.java index 4aadaecafae00..8a700f50464d7 100644 --- a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileMonitoringTest.java +++ b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileMonitoringTest.java @@ -237,26 +237,36 @@ public void testFilePathFiltering() throws Exception { @Test public void testFileSplitMonitoringReprocessWithAppended() throws Exception { - Set uniqFilesFound = new HashSet<>(); + final Set uniqFilesFound = new HashSet<>(); FileCreator fc = new FileCreator(INTERVAL, NO_OF_FILES); fc.start(); - TextInputFormat format = new TextInputFormat(new Path(hdfsURI)); - format.setFilesFilter(FilePathFilter.createDefaultFilter()); - ContinuousFileMonitoringFunction monitoringFunction = - new ContinuousFileMonitoringFunction<>(format, hdfsURI, - FileProcessingMode.PROCESS_CONTINUOUSLY, 1, INTERVAL); - - monitoringFunction.open(new Configuration()); - monitoringFunction.run(new TestingSourceContext(monitoringFunction, uniqFilesFound)); + Thread t = new Thread(new Runnable() { + @Override + public void run() { + TextInputFormat format = new TextInputFormat(new Path(hdfsURI)); + format.setFilesFilter(FilePathFilter.createDefaultFilter()); + ContinuousFileMonitoringFunction monitoringFunction = + new ContinuousFileMonitoringFunction<>(format, hdfsURI, + FileProcessingMode.PROCESS_CONTINUOUSLY, 1, INTERVAL); + + try { + monitoringFunction.open(new Configuration()); + monitoringFunction.run(new TestingSourceContext(monitoringFunction, uniqFilesFound)); + } catch (Exception e) { + // do nothing as we interrupted the thread. + } + } + }); + t.start(); // wait until the sink also sees all the splits. synchronized (uniqFilesFound) { - while (uniqFilesFound.size() < NO_OF_FILES) { - uniqFilesFound.wait(7 * INTERVAL); - } + uniqFilesFound.wait(); } + t.interrupt(); + fc.join(); Assert.assertTrue(fc.getFilesCreated().size() == NO_OF_FILES); Assert.assertTrue(uniqFilesFound.size() == NO_OF_FILES); @@ -281,13 +291,15 @@ public void testFileSplitMonitoringProcessOnce() throws Exception { Set uniqFilesFound = new HashSet<>(); FileCreator fc = new FileCreator(INTERVAL, 1); + Set filesCreated = fc.getFilesCreated(); fc.start(); // to make sure that at least one file is created - Set filesCreated = fc.getFilesCreated(); - synchronized (filesCreated) { - if (filesCreated.size() == 0) { - filesCreated.wait(); + if (filesCreated.size() == 0) { + synchronized (filesCreated) { + if (filesCreated.size() == 0) { + filesCreated.wait(); + } } } Assert.assertTrue(fc.getFilesCreated().size() >= 1); @@ -391,17 +403,17 @@ public void collect(FileInputSplit element) { Assert.fail("Duplicate file: " + filePath); } - filesFound.add(filePath); - try { - if (filesFound.size() == NO_OF_FILES) { - this.src.cancel(); - this.src.close(); - synchronized (filesFound) { + synchronized (filesFound) { + filesFound.add(filePath); + try { + if (filesFound.size() == NO_OF_FILES) { + this.src.cancel(); + this.src.close(); filesFound.notifyAll(); } + } catch (Exception e) { + e.printStackTrace(); } - } catch (Exception e) { - e.printStackTrace(); } } From cc49318468b7191974ac86a672153edcb2387ebb Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 1 Sep 2016 11:41:10 +0200 Subject: [PATCH 008/299] [hotfix] [test-stability] Properly fail if ZooKeeperTestEnvironment cannot delete ZNodes --- .../flink/runtime/zookeeper/ZooKeeperTestEnvironment.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java index 467706f92b622..bd58515e0f7c0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java @@ -144,18 +144,22 @@ public void deleteAll() throws Exception { for (int i = 0; i < maxAttempts; i++) { try { ZKPaths.deleteChildren(client.getZookeeperClient().getZooKeeper(), path, false); - break; + return; } catch (org.apache.zookeeper.KeeperException.NoNodeException e) { // that seems all right. if one of the children we want to delete is // actually already deleted, that's fine. - break; + return; } catch (KeeperException.ConnectionLossException e) { // Keep retrying Thread.sleep(100); } } + + throw new Exception("Could not clear the ZNodes under " + path + ". ZooKeeper is not in " + + "a clean state."); + } } From 923f58f1f6b35047a257cc63ad11f57816ba1b29 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 1 Sep 2016 11:54:18 +0200 Subject: [PATCH 009/299] [hotfix] [tests] Fix CassandraConnectorITCase on Java 7 profiles --- .../connectors/cassandra/CassandraConnectorITCase.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java b/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java index 4822f91693238..8bb440cdbbf9f 100644 --- a/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java +++ b/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java @@ -211,7 +211,10 @@ public static void startFlink() throws Exception { @AfterClass public static void stopFlink() { - flinkCluster.stop(); + if (flinkCluster != null) { + flinkCluster.stop(); + flinkCluster = null; + } } @AfterClass From 3c5b5d6072e395e31f9b115a9204fdc3671002c6 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Fri, 2 Sep 2016 14:52:42 +0200 Subject: [PATCH 010/299] [FLINK-4570] revert Scalastyle version to 0.5.0 This closes #2462 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index d92944fa87f83..28ff9f1f7b4cc 100644 --- a/pom.xml +++ b/pom.xml @@ -1157,7 +1157,7 @@ under the License. org.scalastyle scalastyle-maven-plugin - 0.8.0 + 0.5.0 From 7f9995a98e493d474133fec9eb4d70b46244542b Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 31 Aug 2016 09:33:46 +0200 Subject: [PATCH 011/299] [FLINK-4455] [FLINK-4424] [networkenv] Make NetworkEnvironment independent of ActorGateway and JobManager association Makes the NetworkEnvironment independent of the JobManager association. This means that the NetworkEnvironment and with it the ConnectionManager is started before the TaskManager actor is executed. Furthermore, the ConnectionManager keeps running even in case of a JobManager disassocation. In the wake of the remodelling this behaviour, the PartitionStateChecker and the ResultPartitionConsumableNotifier which depend on the JobManager association were moved out of the NetworkEnvironment. They are now contained in the SlotEnvironment which will be set up when the TaskManager connects to a JobManager. The SlotEnvironment contains all information related to the associated JobManager. Since all slots are implicitly associated with the JobManager which is the leader, we only create one SlotEnvironment which is shared by all Tasks. Introduce SlotEnvironment to accommodate the PartitionStateChecker and ResultPartitionConsumableNotifier Remove the PartitionStateChecker and the ResultPartitionConsumableNotifier from the NetworkEnvironment. Start the NetworkEnvironment when the TaskManager components are created. Keep the NetworkEnvironment running also when the JobManager is disassociated. Fix CassandraConnectorITCase Remove ExecutionContext from TaskManager; Rename SlotEnvironment into JobManagerConnection Introduce JobManagerCommunicationFactory to generate job manager specific communication components This closes #2449. --- .../instance/InstanceConnectionInfo.java | 7 +- .../runtime/io/network/ConnectionID.java | 2 + .../runtime/io/network/ConnectionManager.java | 2 + .../io/network/LocalConnectionManager.java | 5 + .../io/network/NetworkEnvironment.java | 514 ++++-------------- .../runtime/io/network/netty/NettyConfig.java | 2 +- .../network/netty/NettyConnectionManager.java | 9 + .../runtime/io/network/netty/NettyServer.java | 10 + .../io/network/partition/ResultPartition.java | 25 +- .../partition/ResultPartitionManager.java | 2 + .../partition/consumer/InputChannel.java | 8 +- .../partition/consumer/LocalInputChannel.java | 8 +- .../consumer/RemoteInputChannel.java | 8 +- .../partition/consumer/SingleInputGate.java | 32 +- .../consumer/UnknownInputChannel.java | 17 +- .../flink/runtime/query/KvStateRegistry.java | 7 + ...GatewayJobManagerCommunicationFactory.java | 61 +++ .../ActorGatewayKvStateRegistryListener.java | 82 +++ .../ActorGatewayPartitionStateChecker.java | 59 ++ ...ewayResultPartitionConsumableNotifier.java | 82 +++ .../JobManagerCommunicationFactory.java | 47 ++ .../flink/runtime/taskmanager/Task.java | 28 +- .../flink/runtime/jobmanager/JobManager.scala | 6 + .../NetworkEnvironmentConfiguration.scala | 3 +- .../runtime/taskmanager/TaskManager.scala | 152 ++++-- .../testingUtils/TestingTaskManagerLike.scala | 2 +- .../io/network/NetworkEnvironmentTest.java | 189 ++----- .../partition/consumer/InputChannelTest.java | 15 +- .../consumer/LocalInputChannelTest.java | 15 +- .../consumer/RemoteInputChannelTest.java | 15 +- .../consumer/SingleInputGateTest.java | 35 +- .../taskmanager/TaskAsyncCallTest.java | 29 +- ...kManagerComponentsStartupShutdownTest.java | 46 +- .../TaskManagerRegistrationTest.java | 69 --- .../taskmanager/TaskManagerStartupTest.java | 42 +- .../runtime/taskmanager/TaskManagerTest.java | 3 - .../runtime/taskmanager/TaskStopTest.java | 18 +- .../flink/runtime/taskmanager/TaskTest.java | 25 +- .../tasks/InterruptSensitiveRestoreTest.java | 30 +- .../runtime/tasks/StreamTaskTest.java | 35 +- .../test/checkpointing/RescalingITCase.java | 14 +- .../apache/flink/yarn/YarnTaskManager.scala | 6 +- 42 files changed, 932 insertions(+), 834 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayJobManagerCommunicationFactory.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayKvStateRegistryListener.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayPartitionStateChecker.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayResultPartitionConsumableNotifier.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/JobManagerCommunicationFactory.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceConnectionInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceConnectionInfo.java index eb87292afa8ce..2830f04043b28 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceConnectionInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceConnectionInfo.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.instance; import java.io.IOException; +import java.io.Serializable; import java.net.InetAddress; import java.net.UnknownHostException; @@ -36,7 +37,7 @@ * for data exchange. This class also contains utilities to work with the * TaskManager's host name, which is used to localize work assignments. */ -public class InstanceConnectionInfo implements IOReadableWritable, Comparable, java.io.Serializable { +public class InstanceConnectionInfo implements IOReadableWritable, Comparable, Serializable { private static final long serialVersionUID = -8254407801276350716L; @@ -77,7 +78,9 @@ public InstanceConnectionInfo(InetAddress inetAddress, int dataPort) { if (inetAddress == null) { throw new IllegalArgumentException("Argument inetAddress must not be null"); } - if (dataPort <= 0) { + + // -1 indicates a local instance connection info + if (dataPort != -1 && dataPort <= 0) { throw new IllegalArgumentException("Argument dataPort must be greater than zero"); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionID.java index c15e72e73457a..0569daee9d90d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionID.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionID.java @@ -37,6 +37,8 @@ */ public class ConnectionID implements Serializable { + private static final long serialVersionUID = -8068626194818666857L; + private final InetSocketAddress address; private final int connectionIndex; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionManager.java index 2f535fe9527c0..02deb9d2ff8c2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionManager.java @@ -46,6 +46,8 @@ void start(ResultPartitionProvider partitionProvider, int getNumberOfActiveConnections(); + int getDataPort(); + void shutdown() throws IOException; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/LocalConnectionManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/LocalConnectionManager.java index 410f8abe2c529..4f51a56ae2ee2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/LocalConnectionManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/LocalConnectionManager.java @@ -47,6 +47,11 @@ public int getNumberOfActiveConnections() { return 0; } + @Override + public int getDataPort() { + return -1; + } + @Override public void shutdown() {} } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java index 844bc2dadea2c..b221ec71080cb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java @@ -18,60 +18,34 @@ package org.apache.flink.runtime.io.network; -import akka.dispatch.OnFailure; import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; -import org.apache.flink.runtime.instance.ActorGateway; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.io.network.buffer.BufferPool; import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; -import org.apache.flink.runtime.io.network.netty.NettyConfig; -import org.apache.flink.runtime.io.network.netty.NettyConnectionManager; -import org.apache.flink.runtime.io.network.netty.PartitionStateChecker; import org.apache.flink.runtime.io.network.partition.ResultPartition; import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier; -import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate; -import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.runtime.messages.JobManagerMessages.RequestPartitionState; -import org.apache.flink.runtime.messages.TaskMessages.FailTask; -import org.apache.flink.runtime.query.KvStateID; -import org.apache.flink.runtime.query.KvStateMessage; import org.apache.flink.runtime.query.KvStateRegistry; -import org.apache.flink.runtime.query.KvStateRegistryListener; -import org.apache.flink.runtime.query.KvStateServerAddress; import org.apache.flink.runtime.query.TaskKvStateRegistry; -import org.apache.flink.runtime.query.netty.DisabledKvStateRequestStats; import org.apache.flink.runtime.query.netty.KvStateServer; -import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.runtime.taskmanager.TaskManager; import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.Option; -import scala.Tuple2; -import scala.concurrent.ExecutionContext; -import scala.concurrent.Future; -import scala.concurrent.duration.FiniteDuration; import java.io.IOException; -import static org.apache.flink.runtime.messages.JobManagerMessages.ScheduleOrUpdateConsumers; import static org.apache.flink.util.Preconditions.checkNotNull; /** * Network I/O components of each {@link TaskManager} instance. The network environment contains * the data structures that keep track of all intermediate results and all data exchanges. - * - * When initialized, the NetworkEnvironment will allocate the network buffer pool. - * All other components (netty, intermediate result managers, ...) are only created once the - * environment is "associated" with a TaskManager and JobManager. This happens as soon as the - * TaskManager actor gets created and registers itself at the JobManager. */ public class NetworkEnvironment { @@ -79,69 +53,61 @@ public class NetworkEnvironment { private final Object lock = new Object(); - private final NetworkEnvironmentConfiguration configuration; - - private final FiniteDuration jobManagerTimeout; - private final NetworkBufferPool networkBufferPool; - private ConnectionManager connectionManager; + private final ConnectionManager connectionManager; - private ResultPartitionManager partitionManager; + private final ResultPartitionManager resultPartitionManager; - private TaskEventDispatcher taskEventDispatcher; - - private ResultPartitionConsumableNotifier partitionConsumableNotifier; - - private PartitionStateChecker partitionStateChecker; + private final TaskEventDispatcher taskEventDispatcher; /** Server for {@link org.apache.flink.runtime.state.KvState} requests. */ - private KvStateServer kvStateServer; + private final KvStateServer kvStateServer; /** Registry for {@link org.apache.flink.runtime.state.KvState} instances. */ - private KvStateRegistry kvStateRegistry; + private final KvStateRegistry kvStateRegistry; - private boolean isShutdown; + private final IOManager.IOMode defaultIOMode; - /** - * ExecutionEnvironment which is used to execute remote calls with the - * {@link JobManagerResultPartitionConsumableNotifier} - */ - private final ExecutionContext executionContext; + private final int partitionRequestInitialBackoff; - private final InstanceConnectionInfo connectionInfo; + private final int partitionRequestMaxBackoff; + + private boolean isShutdown; - /** - * Initializes all network I/O components. - */ public NetworkEnvironment( - ExecutionContext executionContext, - FiniteDuration jobManagerTimeout, - NetworkEnvironmentConfiguration config, - InstanceConnectionInfo connectionInfo) throws IOException { - - this.executionContext = executionContext; - this.configuration = checkNotNull(config); - this.jobManagerTimeout = checkNotNull(jobManagerTimeout); - this.connectionInfo = checkNotNull(connectionInfo); - - // create the network buffers - this is the operation most likely to fail upon - // mis-configuration, so we do this first - try { - networkBufferPool = new NetworkBufferPool(config.numNetworkBuffers(), - config.networkBufferSize(), config.memoryType()); - } - catch (Throwable t) { - throw new IOException("Cannot allocate network buffer pool: " + t.getMessage(), t); - } + NetworkBufferPool networkBufferPool, + ConnectionManager connectionManager, + ResultPartitionManager resultPartitionManager, + TaskEventDispatcher taskEventDispatcher, + KvStateRegistry kvStateRegistry, + KvStateServer kvStateServer, + IOMode defaultIOMode, + int partitionRequestInitialBackoff, + int partitionRequestMaxBackoff) { + + this.networkBufferPool = checkNotNull(networkBufferPool); + this.connectionManager = checkNotNull(connectionManager); + this.resultPartitionManager = checkNotNull(resultPartitionManager); + this.taskEventDispatcher = checkNotNull(taskEventDispatcher); + this.kvStateRegistry = checkNotNull(kvStateRegistry); + + this.kvStateServer = kvStateServer; + + this.defaultIOMode = defaultIOMode; + + this.partitionRequestInitialBackoff = partitionRequestInitialBackoff; + this.partitionRequestMaxBackoff = partitionRequestMaxBackoff; + + isShutdown = false; } // -------------------------------------------------------------------------------------------- // Properties // -------------------------------------------------------------------------------------------- - public ResultPartitionManager getPartitionManager() { - return partitionManager; + public ResultPartitionManager getResultPartitionManager() { + return resultPartitionManager; } public TaskEventDispatcher getTaskEventDispatcher() { @@ -157,187 +123,27 @@ public NetworkBufferPool getNetworkBufferPool() { } public IOMode getDefaultIOMode() { - return configuration.ioMode(); + return defaultIOMode; } - public ResultPartitionConsumableNotifier getPartitionConsumableNotifier() { - return partitionConsumableNotifier; + public int getPartitionRequestInitialBackoff() { + return partitionRequestInitialBackoff; } - public PartitionStateChecker getPartitionStateChecker() { - return partitionStateChecker; + public int getPartitionRequestMaxBackoff() { + return partitionRequestMaxBackoff; } - public Tuple2 getPartitionRequestInitialAndMaxBackoff() { - return configuration.partitionRequestInitialAndMaxBackoff(); + public KvStateRegistry getKvStateRegistry() { + return kvStateRegistry; } - public TaskKvStateRegistry createKvStateTaskRegistry(JobID jobId, JobVertexID jobVertexId) { - return kvStateRegistry.createTaskRegistry(jobId, jobVertexId); + public KvStateServer getKvStateServer() { + return kvStateServer; } - // -------------------------------------------------------------------------------------------- - // Association / Disassociation with JobManager / TaskManager - // -------------------------------------------------------------------------------------------- - - public boolean isAssociated() { - return partitionConsumableNotifier != null; - } - - /** - * This associates the network environment with a TaskManager and JobManager. - * This will actually start the network components. - * - * @param jobManagerGateway Gateway to the JobManager. - * @param taskManagerGateway Gateway to the TaskManager. - * - * @throws IOException Thrown if the network subsystem (Netty) cannot be properly started. - */ - public void associateWithTaskManagerAndJobManager( - ActorGateway jobManagerGateway, - ActorGateway taskManagerGateway) throws IOException - { - checkNotNull(jobManagerGateway); - checkNotNull(taskManagerGateway); - - synchronized (lock) { - if (isShutdown) { - throw new IllegalStateException("environment is shut down"); - } - - if (this.partitionConsumableNotifier == null && - this.partitionManager == null && - this.taskEventDispatcher == null && - this.connectionManager == null && - this.kvStateRegistry == null && - this.kvStateServer == null) - { - // good, not currently associated. start the individual components - - LOG.debug("Starting result partition manager and network connection manager"); - this.partitionManager = new ResultPartitionManager(); - this.taskEventDispatcher = new TaskEventDispatcher(); - this.partitionConsumableNotifier = new JobManagerResultPartitionConsumableNotifier( - executionContext, - jobManagerGateway, - taskManagerGateway, - jobManagerTimeout); - - this.partitionStateChecker = new JobManagerPartitionStateChecker( - jobManagerGateway, taskManagerGateway); - - // ----- Network connections ----- - final Option nettyConfig = configuration.nettyConfig(); - connectionManager = nettyConfig.isDefined() ? new NettyConnectionManager(nettyConfig.get()) - : new LocalConnectionManager(); - - try { - LOG.debug("Starting network connection manager"); - connectionManager.start(partitionManager, taskEventDispatcher, networkBufferPool); - } - catch (Throwable t) { - throw new IOException("Failed to instantiate network connection manager: " + t.getMessage(), t); - } - - try { - kvStateRegistry = new KvStateRegistry(); - - if (nettyConfig.isDefined()) { - int numNetworkThreads = configuration.queryServerNetworkThreads(); - if (numNetworkThreads == 0) { - numNetworkThreads = nettyConfig.get().getNumberOfSlots(); - } - - int numQueryThreads = configuration.queryServerNetworkThreads(); - if (numQueryThreads == 0) { - numQueryThreads = nettyConfig.get().getNumberOfSlots(); - } - - kvStateServer = new KvStateServer( - connectionInfo.address(), - configuration.queryServerPort(), - numNetworkThreads, - numQueryThreads, - kvStateRegistry, - new DisabledKvStateRequestStats()); - - kvStateServer.start(); - - KvStateRegistryListener listener = new JobManagerKvStateRegistryListener( - jobManagerGateway, - kvStateServer.getAddress()); - - kvStateRegistry.registerListener(listener); - } - } catch (Throwable t) { - throw new IOException("Failed to instantiate KvState management components: " - + t.getMessage(), t); - } - } - else { - throw new IllegalStateException( - "Network Environment is already associated with a JobManager/TaskManager"); - } - } - } - - public void disassociate() throws IOException { - synchronized (lock) { - if (!isAssociated()) { - return; - } - - LOG.debug("Disassociating NetworkEnvironment from TaskManager. Cleaning all intermediate results."); - - // Shut down KvStateRegistry - kvStateRegistry = null; - - // Shut down KvStateServer - if (kvStateServer != null) { - try { - kvStateServer.shutDown(); - } catch (Throwable t) { - throw new IOException("Cannot shutdown KvStateNettyServer", t); - } - kvStateServer = null; - } - - // terminate all network connections - if (connectionManager != null) { - try { - LOG.debug("Shutting down network connection manager"); - connectionManager.shutdown(); - connectionManager = null; - } - catch (Throwable t) { - throw new IOException("Cannot shutdown network connection manager", t); - } - } - - // shutdown all intermediate results - if (partitionManager != null) { - try { - LOG.debug("Shutting down intermediate result partition manager"); - partitionManager.shutdown(); - partitionManager = null; - } - catch (Throwable t) { - throw new IOException("Cannot shutdown partition manager", t); - } - } - - partitionConsumableNotifier = null; - - partitionStateChecker = null; - - if (taskEventDispatcher != null) { - taskEventDispatcher.clearAll(); - taskEventDispatcher = null; - } - - // make sure that the global buffer pool re-acquires all buffers - networkBufferPool.destroyAllBufferPools(); - } + public TaskKvStateRegistry createKvStateTaskRegistry(JobID jobId, JobVertexID jobVertexId) { + return kvStateRegistry.createTaskRegistry(jobId, jobVertexId); } // -------------------------------------------------------------------------------------------- @@ -358,9 +164,6 @@ public void registerTask(Task task) throws IOException { if (isShutdown) { throw new IllegalStateException("NetworkEnvironment is shut down"); } - if (!isAssociated()) { - throw new IllegalStateException("NetworkEnvironment is not associated with a TaskManager"); - } for (int i = 0; i < producedPartitions.length; i++) { final ResultPartition partition = producedPartitions[i]; @@ -373,17 +176,15 @@ public void registerTask(Task task) throws IOException { bufferPool = networkBufferPool.createBufferPool(partition.getNumberOfSubpartitions(), false); partition.registerBufferPool(bufferPool); - partitionManager.registerResultPartition(partition); - } - catch (Throwable t) { + resultPartitionManager.registerResultPartition(partition); + } catch (Throwable t) { if (bufferPool != null) { bufferPool.lazyDestroy(); } if (t instanceof IOException) { throw (IOException) t; - } - else { + } else { throw new IOException(t.getMessage(), t); } } @@ -401,31 +202,18 @@ public void registerTask(Task task) throws IOException { try { bufferPool = networkBufferPool.createBufferPool(gate.getNumberOfInputChannels(), false); gate.setBufferPool(bufferPool); - } - catch (Throwable t) { + } catch (Throwable t) { if (bufferPool != null) { bufferPool.lazyDestroy(); } if (t instanceof IOException) { throw (IOException) t; - } - else { + } else { throw new IOException(t.getMessage(), t); } } } - - // Copy the reference to prevent races with concurrent shut downs - jobManagerNotifier = partitionConsumableNotifier; - } - - for (ResultPartition partition : producedPartitions) { - // Eagerly notify consumers if required. - if (partition.getEagerlyDeployConsumers()) { - jobManagerNotifier.notifyPartitionConsumable( - partition.getJobId(), partition.getPartitionId()); - } } } @@ -436,13 +224,13 @@ public void unregisterTask(Task task) { final ExecutionAttemptID executionId = task.getExecutionId(); synchronized (lock) { - if (isShutdown || !isAssociated()) { + if (isShutdown) { // no need to do anything when we are not operational return; } if (task.isCanceledOrFailed()) { - partitionManager.releasePartitionsProducedBy(executionId, task.getFailureCause()); + resultPartitionManager.releasePartitionsProducedBy(executionId, task.getFailureCause()); } ResultPartitionWriter[] writers = task.getAllWriters(); @@ -476,6 +264,31 @@ public void unregisterTask(Task task) { } } + public void start() throws IOException { + synchronized (lock) { + Preconditions.checkState(!isShutdown, "The NetworkEnvironment has already been shut down."); + + LOG.info("Starting the network environment and its components."); + + try { + LOG.debug("Starting network connection manager"); + connectionManager.start(resultPartitionManager, taskEventDispatcher, networkBufferPool); + } + catch (IOException t) { + throw new IOException("Failed to instantiate network connection manager.", t); + } + + if (kvStateServer != null) { + try { + LOG.debug("Starting the KvState server."); + kvStateServer.start(); + } catch (InterruptedException ie) { + throw new IOException("Failed to start the KvState server.", ie); + } + } + } + } + /** * Tries to shut down all network I/O components. */ @@ -485,20 +298,45 @@ public void shutdown() { return; } - // shut down all connections and free all intermediate result partitions + LOG.info("Shutting down the network environment and its components."); + + if (kvStateServer != null) { + try { + kvStateServer.shutDown(); + } catch (Throwable t) { + LOG.warn("Cannot shut down KvState server.", t); + } + } + + // terminate all network connections try { - disassociate(); + LOG.debug("Shutting down network connection manager"); + connectionManager.shutdown(); } catch (Throwable t) { - LOG.warn("Network services did not shut down properly: " + t.getMessage(), t); + LOG.warn("Cannot shut down the network connection manager.", t); } + // shutdown all intermediate results + try { + LOG.debug("Shutting down intermediate result partition manager"); + resultPartitionManager.shutdown(); + } + catch (Throwable t) { + LOG.warn("Cannot shut down the result partition manager.", t); + } + + taskEventDispatcher.clearAll(); + + // make sure that the global buffer pool re-acquires all buffers + networkBufferPool.destroyAllBufferPools(); + // destroy the buffer pool try { networkBufferPool.destroy(); } catch (Throwable t) { - LOG.warn("Network buffer pool did not shut down properly: " + t.getMessage(), t); + LOG.warn("Network buffer pool did not shut down properly.", t); } isShutdown = true; @@ -506,138 +344,8 @@ public void shutdown() { } public boolean isShutdown() { - return isShutdown; - } - - /** - * Notifies the job manager about consumable partitions. - */ - private static class JobManagerResultPartitionConsumableNotifier implements ResultPartitionConsumableNotifier { - - /** - * {@link ExecutionContext} which is used for the failure handler of {@link ScheduleOrUpdateConsumers} - * messages. - */ - private final ExecutionContext executionContext; - - private final ActorGateway jobManager; - - private final ActorGateway taskManager; - - private final FiniteDuration jobManagerMessageTimeout; - - public JobManagerResultPartitionConsumableNotifier( - ExecutionContext executionContext, - ActorGateway jobManager, - ActorGateway taskManager, - FiniteDuration jobManagerMessageTimeout) { - - this.executionContext = executionContext; - this.jobManager = jobManager; - this.taskManager = taskManager; - this.jobManagerMessageTimeout = jobManagerMessageTimeout; - } - - @Override - public void notifyPartitionConsumable(JobID jobId, final ResultPartitionID partitionId) { - - final ScheduleOrUpdateConsumers msg = new ScheduleOrUpdateConsumers(jobId, partitionId); - - Future futureResponse = jobManager.ask(msg, jobManagerMessageTimeout); - - futureResponse.onFailure(new OnFailure() { - @Override - public void onFailure(Throwable failure) { - LOG.error("Could not schedule or update consumers at the JobManager.", failure); - - // Fail task at the TaskManager - FailTask failMsg = new FailTask( - partitionId.getProducerId(), - new RuntimeException("Could not notify JobManager to schedule or update consumers", - failure)); - - taskManager.tell(failMsg); - } - }, executionContext); - } - } - - private static class JobManagerPartitionStateChecker implements PartitionStateChecker { - - private final ActorGateway jobManager; - - private final ActorGateway taskManager; - - public JobManagerPartitionStateChecker(ActorGateway jobManager, ActorGateway taskManager) { - this.jobManager = jobManager; - this.taskManager = taskManager; - } - - @Override - public void triggerPartitionStateCheck( - JobID jobId, - ExecutionAttemptID executionAttemptID, - IntermediateDataSetID resultId, - ResultPartitionID partitionId) { - - RequestPartitionState msg = new RequestPartitionState( - jobId, partitionId, executionAttemptID, resultId); - - jobManager.tell(msg, taskManager); - } - } - - /** - * Simple {@link KvStateRegistry} listener, which forwards registrations to - * the JobManager. - */ - private static class JobManagerKvStateRegistryListener implements KvStateRegistryListener { - - private ActorGateway jobManager; - - private KvStateServerAddress kvStateServerAddress; - - public JobManagerKvStateRegistryListener( - ActorGateway jobManager, - KvStateServerAddress kvStateServerAddress) { - - this.jobManager = Preconditions.checkNotNull(jobManager, "JobManager"); - this.kvStateServerAddress = Preconditions.checkNotNull(kvStateServerAddress, "KvStateServerAddress"); - } - - @Override - public void notifyKvStateRegistered( - JobID jobId, - JobVertexID jobVertexId, - int keyGroupIndex, - String registrationName, - KvStateID kvStateId) { - - Object msg = new KvStateMessage.NotifyKvStateRegistered( - jobId, - jobVertexId, - keyGroupIndex, - registrationName, - kvStateId, - kvStateServerAddress); - - jobManager.tell(msg); - } - - @Override - public void notifyKvStateUnregistered( - JobID jobId, - JobVertexID jobVertexId, - int keyGroupIndex, - String registrationName) { - - Object msg = new KvStateMessage.NotifyKvStateUnregistered( - jobId, - jobVertexId, - keyGroupIndex, - registrationName); - - jobManager.tell(msg); + synchronized (lock) { + return isShutdown; } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConfig.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConfig.java index 68061362f6322..c178f2ece1bad 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConfig.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConfig.java @@ -76,7 +76,7 @@ public NettyConfig( this.serverAddress = checkNotNull(serverAddress); - checkArgument(serverPort > 0 && serverPort <= 65536, "Invalid port number."); + checkArgument(serverPort >= 0 && serverPort <= 65536, "Invalid port number."); this.serverPort = serverPort; checkArgument(memorySegmentSize > 0, "Invalid memory segment size."); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConnectionManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConnectionManager.java index d278b3c7effe1..abee2a8fb1b02 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConnectionManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConnectionManager.java @@ -70,6 +70,15 @@ public int getNumberOfActiveConnections() { return partitionRequestClientFactory.getNumberOfActiveClients(); } + @Override + public int getDataPort() { + if (server != null && server.getLocalAddress() != null) { + return server.getLocalAddress().getPort(); + } else { + return -1; + } + } + @Override public void shutdown() { client.shutdown(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyServer.java index 036fe22ccf3f5..a93e90c7c4cd6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyServer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyServer.java @@ -33,6 +33,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.net.InetSocketAddress; import java.util.concurrent.ThreadFactory; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -50,8 +51,11 @@ class NettyServer { private ChannelFuture bindFuture; + private InetSocketAddress localAddress; + NettyServer(NettyConfig config) { this.config = checkNotNull(config); + localAddress = null; } void init(final NettyProtocol protocol, NettyBufferPool nettyBufferPool) throws IOException { @@ -128,6 +132,8 @@ public void initChannel(SocketChannel channel) throws Exception { bindFuture = bootstrap.bind().syncUninterruptibly(); + localAddress = (InetSocketAddress) bindFuture.channel().localAddress(); + long end = System.currentTimeMillis(); LOG.info("Successful initialization (took {} ms). Listening on SocketAddress {}.", (end - start), bindFuture.channel().localAddress().toString()); } @@ -140,6 +146,10 @@ ServerBootstrap getBootstrap() { return bootstrap; } + public InetSocketAddress getLocalAddress() { + return localAddress; + } + void shutdown() { long start = System.currentTimeMillis(); if (bindFuture != null) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java index 7c109f3a57f71..7bcdd31910118 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java @@ -92,7 +92,7 @@ public class ResultPartition implements BufferPoolOwner { *

If true, the consumers are deployed as soon as the * runtime result is registered at the result manager of the task manager. */ - private final boolean eagerlyDeployConsumers; + private final boolean doEagerDeployment; /** The subpartitions of this partition. At least one. */ private final ResultSubpartition[] subpartitions; @@ -133,7 +133,7 @@ public ResultPartition( JobID jobId, ResultPartitionID partitionId, ResultPartitionType partitionType, - boolean eagerlyDeployConsumers, + boolean doEagerDeployment, int numberOfSubpartitions, ResultPartitionManager partitionManager, ResultPartitionConsumableNotifier partitionConsumableNotifier, @@ -144,7 +144,7 @@ public ResultPartition( this.jobId = checkNotNull(jobId); this.partitionId = checkNotNull(partitionId); this.partitionType = checkNotNull(partitionType); - this.eagerlyDeployConsumers = eagerlyDeployConsumers; + this.doEagerDeployment = doEagerDeployment; this.subpartitions = new ResultSubpartition[numberOfSubpartitions]; this.partitionManager = checkNotNull(partitionManager); this.partitionConsumableNotifier = checkNotNull(partitionConsumableNotifier); @@ -211,16 +211,6 @@ public int getNumberOfSubpartitions() { return subpartitions.length; } - /** - * Returns whether consumers should be deployed eagerly (as soon as they - * are registered at the result manager of the task manager). - * - * @return Whether consumers should be deployed eagerly - */ - public boolean getEagerlyDeployConsumers() { - return eagerlyDeployConsumers; - } - public BufferProvider getBufferProvider() { return bufferPool; } @@ -356,6 +346,15 @@ public Throwable getFailureCause() { return cause; } + /** + * Deploys consumers if eager deployment is activated + */ + public void deployConsumers() { + if (doEagerDeployment) { + partitionConsumableNotifier.notifyPartitionConsumable(jobId, partitionId); + } + } + /** * Releases buffers held by this result partition. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java index 9da3e14a5097a..6edae6fbf3856 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java @@ -58,6 +58,8 @@ public void registerResultPartition(ResultPartition partition) throws IOExceptio throw new IllegalStateException("Result partition already registered."); } + partition.deployConsumers(); + LOG.debug("Registered {}.", partition); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java index 5d82903efc7b9..35094e248e578 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java @@ -24,7 +24,6 @@ import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView; -import scala.Tuple2; import java.io.IOException; import java.util.concurrent.atomic.AtomicReference; @@ -71,13 +70,14 @@ protected InputChannel( SingleInputGate inputGate, int channelIndex, ResultPartitionID partitionId, - Tuple2 initialAndMaxBackoff, + int initialBackoff, + int maxBackoff, Counter numBytesIn) { checkArgument(channelIndex >= 0); - int initial = initialAndMaxBackoff._1(); - int max = initialAndMaxBackoff._2(); + int initial = initialBackoff; + int max = maxBackoff; checkArgument(initial >= 0 && initial <= max); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java index 6fcd2f9753187..a8aae7edb534d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java @@ -32,7 +32,6 @@ import org.apache.flink.runtime.util.event.NotificationListener; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.Tuple2; import java.io.IOException; import java.util.Timer; @@ -72,7 +71,7 @@ public class LocalInputChannel extends InputChannel implements NotificationListe IOMetricGroup metrics) { this(inputGate, channelIndex, partitionId, partitionManager, taskEventDispatcher, - new Tuple2(0, 0), metrics); + 0, 0, metrics); } LocalInputChannel( @@ -81,10 +80,11 @@ public class LocalInputChannel extends InputChannel implements NotificationListe ResultPartitionID partitionId, ResultPartitionManager partitionManager, TaskEventDispatcher taskEventDispatcher, - Tuple2 initialAndMaxBackoff, + int initialBackoff, + int maxBackoff, IOMetricGroup metrics) { - super(inputGate, channelIndex, partitionId, initialAndMaxBackoff, metrics.getNumBytesInLocalCounter()); + super(inputGate, channelIndex, partitionId, initialBackoff, maxBackoff, metrics.getNumBytesInLocalCounter()); this.partitionManager = checkNotNull(partitionManager); this.taskEventDispatcher = checkNotNull(taskEventDispatcher); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java index 1cd042c4f7c5d..a12d2a8f3deca 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java @@ -29,7 +29,6 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.Tuple2; import java.io.IOException; import java.util.ArrayDeque; @@ -85,7 +84,7 @@ public RemoteInputChannel( IOMetricGroup metrics) { this(inputGate, channelIndex, partitionId, connectionId, connectionManager, - new Tuple2(0, 0), metrics); + 0, 0, metrics); } public RemoteInputChannel( @@ -94,10 +93,11 @@ public RemoteInputChannel( ResultPartitionID partitionId, ConnectionID connectionId, ConnectionManager connectionManager, - Tuple2 initialAndMaxBackoff, + int initialBackoff, + int maxBackoff, IOMetricGroup metrics) { - super(inputGate, channelIndex, partitionId, initialAndMaxBackoff, metrics.getNumBytesInRemoteCounter()); + super(inputGate, channelIndex, partitionId, initialBackoff, maxBackoff, metrics.getNumBytesInRemoteCounter()); this.connectionId = checkNotNull(connectionId); this.connectionManager = checkNotNull(connectionManager); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java index 351181aa46052..aaf8887fb49c3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java @@ -505,6 +505,7 @@ public static SingleInputGate create( ExecutionAttemptID executionId, InputGateDeploymentDescriptor igdd, NetworkEnvironment networkEnvironment, + PartitionStateChecker partitionStateChecker, IOMetricGroup metrics) { final IntermediateDataSetID consumedResultId = checkNotNull(igdd.getConsumedResultId()); @@ -516,7 +517,7 @@ public static SingleInputGate create( final SingleInputGate inputGate = new SingleInputGate( owningTaskName, jobId, executionId, consumedResultId, consumedSubpartitionIndex, - icdd.length, networkEnvironment.getPartitionStateChecker(), metrics); + icdd.length, partitionStateChecker, metrics); // Create the input channels. There is one input channel for each consumed partition. final InputChannel[] inputChannels = new InputChannel[icdd.length]; @@ -528,27 +529,30 @@ public static SingleInputGate create( if (partitionLocation.isLocal()) { inputChannels[i] = new LocalInputChannel(inputGate, i, partitionId, - networkEnvironment.getPartitionManager(), - networkEnvironment.getTaskEventDispatcher(), - networkEnvironment.getPartitionRequestInitialAndMaxBackoff(), - metrics + networkEnvironment.getResultPartitionManager(), + networkEnvironment.getTaskEventDispatcher(), + networkEnvironment.getPartitionRequestInitialBackoff(), + networkEnvironment.getPartitionRequestMaxBackoff(), + metrics ); } else if (partitionLocation.isRemote()) { inputChannels[i] = new RemoteInputChannel(inputGate, i, partitionId, - partitionLocation.getConnectionId(), - networkEnvironment.getConnectionManager(), - networkEnvironment.getPartitionRequestInitialAndMaxBackoff(), - metrics + partitionLocation.getConnectionId(), + networkEnvironment.getConnectionManager(), + networkEnvironment.getPartitionRequestInitialBackoff(), + networkEnvironment.getPartitionRequestInitialBackoff(), + metrics ); } else if (partitionLocation.isUnknown()) { inputChannels[i] = new UnknownInputChannel(inputGate, i, partitionId, - networkEnvironment.getPartitionManager(), - networkEnvironment.getTaskEventDispatcher(), - networkEnvironment.getConnectionManager(), - networkEnvironment.getPartitionRequestInitialAndMaxBackoff(), - metrics + networkEnvironment.getResultPartitionManager(), + networkEnvironment.getTaskEventDispatcher(), + networkEnvironment.getConnectionManager(), + networkEnvironment.getPartitionRequestInitialBackoff(), + networkEnvironment.getPartitionRequestMaxBackoff(), + metrics ); } else { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java index cc91e836b3a42..27ecc7031c2fb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java @@ -27,7 +27,6 @@ import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; -import scala.Tuple2; import java.io.IOException; @@ -46,7 +45,9 @@ public class UnknownInputChannel extends InputChannel { private final ConnectionManager connectionManager; /** Initial and maximum backoff (in ms) after failed partition requests. */ - private final Tuple2 partitionRequestInitialAndMaxBackoff; + private final int initialBackoff; + + private final int maxBackoff; private final IOMetricGroup metrics; @@ -57,16 +58,18 @@ public UnknownInputChannel( ResultPartitionManager partitionManager, TaskEventDispatcher taskEventDispatcher, ConnectionManager connectionManager, - Tuple2 partitionRequestInitialAndMaxBackoff, + int initialBackoff, + int maxBackoff, IOMetricGroup metrics) { - super(gate, channelIndex, partitionId, partitionRequestInitialAndMaxBackoff, null); + super(gate, channelIndex, partitionId, initialBackoff, maxBackoff, null); this.partitionManager = checkNotNull(partitionManager); this.taskEventDispatcher = checkNotNull(taskEventDispatcher); this.connectionManager = checkNotNull(connectionManager); - this.partitionRequestInitialAndMaxBackoff = checkNotNull(partitionRequestInitialAndMaxBackoff); this.metrics = checkNotNull(metrics); + this.initialBackoff = initialBackoff; + this.maxBackoff = maxBackoff; } @Override @@ -117,10 +120,10 @@ public String toString() { // ------------------------------------------------------------------------ public RemoteInputChannel toRemoteInputChannel(ConnectionID producerAddress) { - return new RemoteInputChannel(inputGate, channelIndex, partitionId, checkNotNull(producerAddress), connectionManager, partitionRequestInitialAndMaxBackoff, metrics); + return new RemoteInputChannel(inputGate, channelIndex, partitionId, checkNotNull(producerAddress), connectionManager, initialBackoff, maxBackoff, metrics); } public LocalInputChannel toLocalInputChannel() { - return new LocalInputChannel(inputGate, channelIndex, partitionId, partitionManager, taskEventDispatcher, partitionRequestInitialAndMaxBackoff, metrics); + return new LocalInputChannel(inputGate, channelIndex, partitionId, partitionManager, taskEventDispatcher, initialBackoff, maxBackoff, metrics); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistry.java index 5213fe9336aa7..f19c123b1080b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistry.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistry.java @@ -58,6 +58,13 @@ public void registerListener(KvStateRegistryListener listener) { } } + /** + * Unregisters the listener with the registry + */ + public void unregisterListener() { + listener.set(null); + } + /** * Registers the KvState instance identified by the given 4-tuple of JobID, * JobVertexID, key group index, and registration name. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayJobManagerCommunicationFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayJobManagerCommunicationFactory.java new file mode 100644 index 0000000000000..4697c79e15301 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayJobManagerCommunicationFactory.java @@ -0,0 +1,61 @@ +/* + * 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.runtime.taskmanager; + +import org.apache.flink.runtime.instance.ActorGateway; +import org.apache.flink.runtime.io.network.netty.PartitionStateChecker; +import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier; +import org.apache.flink.util.Preconditions; +import scala.concurrent.ExecutionContext; +import scala.concurrent.duration.FiniteDuration; + +/** + * Factory implementation which generates {@link ActorGateway} based job manager communication + * components. + */ +public class ActorGatewayJobManagerCommunicationFactory implements JobManagerCommunicationFactory { + private final ExecutionContext executionContext; + private final ActorGateway jobManagerGateway; + private final ActorGateway taskManagerGateway; + private final FiniteDuration jobManagerMessageTimeout; + + public ActorGatewayJobManagerCommunicationFactory( + ExecutionContext executionContext, + ActorGateway jobManagerGateway, + ActorGateway taskManagerGateway, + FiniteDuration jobManagerMessageTimeout) { + + this.executionContext = Preconditions.checkNotNull(executionContext); + this.jobManagerGateway = Preconditions.checkNotNull(jobManagerGateway); + this.taskManagerGateway = Preconditions.checkNotNull(taskManagerGateway); + this.jobManagerMessageTimeout = Preconditions.checkNotNull(jobManagerMessageTimeout); + } + + public PartitionStateChecker createPartitionStateChecker() { + return new ActorGatewayPartitionStateChecker(jobManagerGateway, taskManagerGateway); + } + + public ResultPartitionConsumableNotifier createResultPartitionConsumableNotifier(Task owningTask) { + return new ActorGatewayResultPartitionConsumableNotifier( + executionContext, + jobManagerGateway, + owningTask, + jobManagerMessageTimeout); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayKvStateRegistryListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayKvStateRegistryListener.java new file mode 100644 index 0000000000000..2d6993856d11b --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayKvStateRegistryListener.java @@ -0,0 +1,82 @@ +/* + * 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.runtime.taskmanager; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.instance.ActorGateway; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.query.KvStateID; +import org.apache.flink.runtime.query.KvStateMessage; +import org.apache.flink.runtime.query.KvStateRegistryListener; +import org.apache.flink.runtime.query.KvStateServerAddress; +import org.apache.flink.util.Preconditions; + +/** + * This implementation uses {@link ActorGateway} to forward key-value state notifications to the job + * manager. The notifications are wrapped in an actor message and send to the given actor gateway. + */ +public class ActorGatewayKvStateRegistryListener implements KvStateRegistryListener { + + private ActorGateway jobManager; + + private KvStateServerAddress kvStateServerAddress; + + public ActorGatewayKvStateRegistryListener( + ActorGateway jobManager, + KvStateServerAddress kvStateServerAddress) { + + this.jobManager = Preconditions.checkNotNull(jobManager, "JobManager"); + this.kvStateServerAddress = Preconditions.checkNotNull(kvStateServerAddress, "KvStateServerAddress"); + } + + @Override + public void notifyKvStateRegistered( + JobID jobId, + JobVertexID jobVertexId, + int keyGroupIndex, + String registrationName, + KvStateID kvStateId) { + + Object msg = new KvStateMessage.NotifyKvStateRegistered( + jobId, + jobVertexId, + keyGroupIndex, + registrationName, + kvStateId, + kvStateServerAddress); + + jobManager.tell(msg); + } + + @Override + public void notifyKvStateUnregistered( + JobID jobId, + JobVertexID jobVertexId, + int keyGroupIndex, + String registrationName) { + + Object msg = new KvStateMessage.NotifyKvStateUnregistered( + jobId, + jobVertexId, + keyGroupIndex, + registrationName); + + jobManager.tell(msg); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayPartitionStateChecker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayPartitionStateChecker.java new file mode 100644 index 0000000000000..e7c66901df22b --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayPartitionStateChecker.java @@ -0,0 +1,59 @@ +/* + * 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.runtime.taskmanager; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.instance.ActorGateway; +import org.apache.flink.runtime.io.network.netty.PartitionStateChecker; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.messages.JobManagerMessages; + +/** + * This implementation uses {@link ActorGateway} to trigger the partition state check at the job + * manager. + */ +public class ActorGatewayPartitionStateChecker implements PartitionStateChecker { + + private final ActorGateway jobManager; + + private final ActorGateway taskManager; + + public ActorGatewayPartitionStateChecker(ActorGateway jobManager, ActorGateway taskManager) { + this.jobManager = jobManager; + this.taskManager = taskManager; + } + + @Override + public void triggerPartitionStateCheck( + JobID jobId, + ExecutionAttemptID executionAttemptID, + IntermediateDataSetID resultId, + ResultPartitionID partitionId) { + + JobManagerMessages.RequestPartitionState msg = new JobManagerMessages.RequestPartitionState( + jobId, + partitionId, + executionAttemptID, + resultId); + + jobManager.tell(msg, taskManager); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayResultPartitionConsumableNotifier.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayResultPartitionConsumableNotifier.java new file mode 100644 index 0000000000000..b91120b4d3788 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayResultPartitionConsumableNotifier.java @@ -0,0 +1,82 @@ +/* + * 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.runtime.taskmanager; + +import akka.dispatch.OnFailure; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.instance.ActorGateway; +import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.concurrent.ExecutionContext; +import scala.concurrent.Future; +import scala.concurrent.duration.FiniteDuration; + +/** + * This implementation uses {@link ActorGateway} to notify the job manager about consumable + * partitions. + */ +public class ActorGatewayResultPartitionConsumableNotifier implements ResultPartitionConsumableNotifier { + + private static final Logger LOG = LoggerFactory.getLogger(ActorGatewayResultPartitionConsumableNotifier.class); + + /** + * {@link ExecutionContext} which is used for the failure handler of + * {@link JobManagerMessages.ScheduleOrUpdateConsumers} messages. + */ + private final ExecutionContext executionContext; + + private final ActorGateway jobManager; + + private final Task owningTask; + + private final FiniteDuration jobManagerMessageTimeout; + + public ActorGatewayResultPartitionConsumableNotifier( + ExecutionContext executionContext, + ActorGateway jobManager, + Task owningTask, + FiniteDuration jobManagerMessageTimeout) { + + this.executionContext = Preconditions.checkNotNull(executionContext); + this.jobManager = Preconditions.checkNotNull(jobManager); + this.owningTask = Preconditions.checkNotNull(owningTask); + this.jobManagerMessageTimeout = Preconditions.checkNotNull(jobManagerMessageTimeout); + } + + @Override + public void notifyPartitionConsumable(JobID jobId, final ResultPartitionID partitionId) { + + final JobManagerMessages.ScheduleOrUpdateConsumers msg = new JobManagerMessages.ScheduleOrUpdateConsumers(jobId, partitionId); + + Future futureResponse = jobManager.ask(msg, jobManagerMessageTimeout); + + futureResponse.onFailure(new OnFailure() { + @Override + public void onFailure(Throwable failure) { + LOG.error("Could not schedule or update consumers at the JobManager.", failure); + + owningTask.failExternally(new RuntimeException("Could not notify JobManager to schedule or update consumers", failure)); + } + }, executionContext); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/JobManagerCommunicationFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/JobManagerCommunicationFactory.java new file mode 100644 index 0000000000000..64cfcb1e62306 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/JobManagerCommunicationFactory.java @@ -0,0 +1,47 @@ +/* + * 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.runtime.taskmanager; + +import org.apache.flink.runtime.io.network.netty.PartitionStateChecker; +import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier; + +/** + * Factory to generate job manager specific communication components. + */ +public interface JobManagerCommunicationFactory { + + /** + * Creates a {@link PartitionStateChecker} which communicates with the associated job manager of + * this instance. + * + * @return PartitionStateChecker which communicates with the associated job manager of this + * instance + */ + PartitionStateChecker createPartitionStateChecker(); + + /** + * Creates a {@link ResultPartitionConsumableNotifier} which communicates with the associated + * job manager of this instance. + * + * @param owningTask Task which is associated with the ResultPartitionConsumableNotifier + * @return ResultPartitionConsumableNotifier which communicates with the associated job manager + * of this instance + */ + ResultPartitionConsumableNotifier createResultPartitionConsumableNotifier(Task owningTask); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java index 73601c488cd1b..d09e03c7c8978 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java @@ -25,6 +25,8 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.io.network.netty.PartitionStateChecker; +import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.blob.BlobKey; @@ -91,10 +93,9 @@ * with the JobManager. * *

The Flink operators (implemented as subclasses of - * {@link org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable} have only data - * readers, -writers, and certain event callbacks. The task connects those to the - * network stack and actor messages, and tracks the state of the execution and - * handles exceptions. + * {@link AbstractInvokable} have only data readers, -writers, and certain event callbacks. + * The task connects those to the network stack and actor messages, and tracks the state + * of the execution and handles exceptions. * *

Tasks have no knowledge about how they relate to other tasks, or whether they * are the first attempt to execute the task, or a repeated attempt. All of that @@ -247,6 +248,7 @@ public Task(TaskDeploymentDescriptor tdd, MemoryManager memManager, IOManager ioManager, NetworkEnvironment networkEnvironment, + JobManagerCommunicationFactory jobManagerCommunicationFactory, BroadcastVariableManager bcVarManager, ActorGateway taskManagerActor, ActorGateway jobManagerActor, @@ -302,6 +304,9 @@ public Task(TaskDeploymentDescriptor tdd, this.producedPartitions = new ResultPartition[partitions.size()]; this.writers = new ResultPartitionWriter[partitions.size()]; + ResultPartitionConsumableNotifier resultPartitionConsumableNotifier = + jobManagerCommunicationFactory.createResultPartitionConsumableNotifier(this); + for (int i = 0; i < this.producedPartitions.length; i++) { ResultPartitionDeploymentDescriptor desc = partitions.get(i); ResultPartitionID partitionId = new ResultPartitionID(desc.getPartitionId(), executionId); @@ -313,8 +318,8 @@ public Task(TaskDeploymentDescriptor tdd, desc.getPartitionType(), desc.getEagerlyDeployConsumers(), desc.getNumberOfSubpartitions(), - networkEnvironment.getPartitionManager(), - networkEnvironment.getPartitionConsumableNotifier(), + networkEnvironment.getResultPartitionManager(), + resultPartitionConsumableNotifier, ioManager, networkEnvironment.getDefaultIOMode()); @@ -325,10 +330,17 @@ public Task(TaskDeploymentDescriptor tdd, this.inputGates = new SingleInputGate[consumedPartitions.size()]; this.inputGatesById = new HashMap(); + PartitionStateChecker partitionStateChecker = jobManagerCommunicationFactory.createPartitionStateChecker(); + for (int i = 0; i < this.inputGates.length; i++) { SingleInputGate gate = SingleInputGate.create( - taskNameWithSubtaskAndId, jobId, executionId, consumedPartitions.get(i), networkEnvironment, - metricGroup.getIOMetricGroup()); + taskNameWithSubtaskAndId, + jobId, + executionId, + consumedPartitions.get(i), + networkEnvironment, + partitionStateChecker, + metricGroup.getIOMetricGroup()); this.inputGates[i] = gate; inputGatesById.put(gate.getConsumedResultId(), gate); diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index 407fa01a8eeb5..0c62c69b002ce 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -1472,6 +1472,9 @@ class JobManager( currentJobs.get(msg.getJobId) match { case Some((graph, _)) => try { + log.debug(s"Lookup key-value state for job ${msg.getJobId} with registration " + + s"name ${msg.getRegistrationName}.") + val registry = graph.getKvStateLocationRegistry val location = registry.getKvStateLocation(msg.getRegistrationName) if (location == null) { @@ -1493,6 +1496,9 @@ class JobManager( currentJobs.get(msg.getJobId) match { case Some((graph, _)) => try { + log.debug(s"Key value state registered for job ${msg.getJobId} under " + + s"name ${msg.getRegistrationName}.") + graph.getKvStateLocationRegistry.notifyKvStateRegistered( msg.getJobVertexId, msg.getKeyGroupIndex, diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/NetworkEnvironmentConfiguration.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/NetworkEnvironmentConfiguration.scala index 0788d7c28d3a4..893eaa8d0a28a 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/NetworkEnvironmentConfiguration.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/NetworkEnvironmentConfiguration.scala @@ -31,4 +31,5 @@ case class NetworkEnvironmentConfiguration( queryServerNetworkThreads: Int, queryServerQueryThreads: Int, nettyConfig: Option[NettyConfig] = None, - partitionRequestInitialAndMaxBackoff: (Integer, Integer) = (500, 3000)) + partitionRequestInitialBackoff: Int = 500, + partitinRequestMaxBackoff: Int = 3000) diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index 72ec2ac4474f7..31548263ff871 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -53,8 +53,10 @@ import org.apache.flink.runtime.filecache.FileCache import org.apache.flink.runtime.instance.{AkkaActorGateway, HardwareDescription, InstanceConnectionInfo, InstanceID} import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode import org.apache.flink.runtime.io.disk.iomanager.{IOManager, IOManagerAsync} -import org.apache.flink.runtime.io.network.NetworkEnvironment -import org.apache.flink.runtime.io.network.netty.NettyConfig +import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool +import org.apache.flink.runtime.io.network.{LocalConnectionManager, NetworkEnvironment, TaskEventDispatcher} +import org.apache.flink.runtime.io.network.netty.{NettyConfig, NettyConnectionManager} +import org.apache.flink.runtime.io.network.partition.ResultPartitionManager import org.apache.flink.runtime.jobgraph.IntermediateDataSetID import org.apache.flink.runtime.leaderretrieval.{LeaderRetrievalListener, LeaderRetrievalService} import org.apache.flink.runtime.memory.MemoryManager @@ -67,6 +69,8 @@ import org.apache.flink.runtime.messages.checkpoint.{AbstractCheckpointMessage, import org.apache.flink.runtime.metrics.{MetricRegistry => FlinkMetricRegistry} import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup import org.apache.flink.runtime.process.ProcessReaper +import org.apache.flink.runtime.query.KvStateRegistry +import org.apache.flink.runtime.query.netty.{DisabledKvStateRequestStats, KvStateServer} import org.apache.flink.runtime.security.SecurityUtils import org.apache.flink.runtime.security.SecurityUtils.FlinkSecuredRunner import org.apache.flink.runtime.util._ @@ -76,7 +80,6 @@ import org.apache.flink.util.{MathUtils, NetUtils} import scala.collection.JavaConverters._ import scala.concurrent._ import scala.concurrent.duration._ -import scala.concurrent.forkjoin.ForkJoinPool import scala.language.postfixOps import scala.util.{Failure, Success} @@ -193,6 +196,8 @@ class TaskManager( private var scheduledTaskManagerRegistration: Option[Cancellable] = None private var currentRegistrationRun: UUID = UUID.randomUUID() + private var jobManagerConnectionFactory: Option[JobManagerCommunicationFactory] = None + // -------------------------------------------------------------------------- // Actor messages and life cycle // -------------------------------------------------------------------------- @@ -395,13 +400,11 @@ class TaskManager( // discards intermediate result partitions of a task execution on this TaskManager case FailIntermediateResultPartitions(executionID) => log.info("Discarding the results produced by task execution " + executionID) - if (network.isAssociated) { - try { - network.getPartitionManager.releasePartitionsProducedBy(executionID) - } catch { - case t: Throwable => killTaskManagerFatal( - "Fatal leak: Unable to release intermediate result partition data", t) - } + try { + network.getResultPartitionManager.releasePartitionsProducedBy(executionID) + } catch { + case t: Throwable => killTaskManagerFatal( + "Fatal leak: Unable to release intermediate result partition data", t) } // notifies the TaskManager that the state of a task has changed. @@ -916,25 +919,33 @@ class TaskManager( "starting network stack and library cache.") // sanity check that the JobManager dependent components are not set up currently - if (network.isAssociated || blobService.isDefined) { + if (jobManagerConnectionFactory.isDefined || blobService.isDefined) { throw new IllegalStateException("JobManager-specific components are already initialized.") } currentJobManager = Some(jobManager) instanceID = id - // start the network stack, now that we have the JobManager actor reference - try { - network.associateWithTaskManagerAndJobManager( - new AkkaActorGateway(jobManager, leaderSessionID.orNull), - new AkkaActorGateway(self, leaderSessionID.orNull) - ) - } - catch { - case e: Exception => - val message = "Could not start network environment." - log.error(message, e) - throw new RuntimeException(message, e) + val jobManagerGateway = new AkkaActorGateway(jobManager, leaderSessionID.orNull) + val taskmanagerGateway = new AkkaActorGateway(self, leaderSessionID.orNull) + + jobManagerConnectionFactory = Some( + new ActorGatewayJobManagerCommunicationFactory( + context.dispatcher, + jobManagerGateway, + taskmanagerGateway, + config.timeout)) + + + val kvStateServer = network.getKvStateServer() + + if (kvStateServer != null) { + val kvStateRegistry = network.getKvStateRegistry() + + kvStateRegistry.registerListener( + new ActorGatewayKvStateRegistryListener( + jobManagerGateway, + kvStateServer.getAddress)) } // start a blob service, if a blob server is specified @@ -1031,8 +1042,12 @@ class TaskManager( } blobService = None - // disassociate the network environment - network.disassociate() + // disassociate the slot environment + jobManagerConnectionFactory = None + + if (network.getKvStateRegistry != null) { + network.getKvStateRegistry.unregisterListener() + } // stop the metrics reporters metricsRegistry.shutdown() @@ -1092,6 +1107,13 @@ class TaskManager( case None => throw new IllegalStateException("There is no valid library cache manager.") } + val jmFactory = jobManagerConnectionFactory match { + case Some(factory) => factory + case None => + throw new IllegalStateException("TaskManager is not associated with a JobManager and, " + + "thus, the SlotEnvironment has not been initialized.") + } + val slot = tdd.getTargetSlotNumber if (slot < 0 || slot >= numberOfSlots) { throw new IllegalArgumentException(s"Target slot $slot does not exist on TaskManager.") @@ -1117,6 +1139,7 @@ class TaskManager( memoryManager, ioManager, network, + jmFactory, bcVarManager, selfGateway, jobManagerGateway, @@ -1796,7 +1819,7 @@ object TaskManager { val (taskManagerConfig : TaskManagerConfiguration, netConfig: NetworkEnvironmentConfiguration, - connectionInfo: InstanceConnectionInfo, + taskManagerAddress: InetSocketAddress, memType: MemoryType ) = parseTaskManagerConfiguration( configuration, @@ -1806,14 +1829,64 @@ object TaskManager { // pre-start checks checkTempDirs(taskManagerConfig.tmpDirPaths) - val executionContext = ExecutionContext.fromExecutor(new ForkJoinPool()) + val networkBufferPool = new NetworkBufferPool( + netConfig.numNetworkBuffers, + netConfig.networkBufferSize, + netConfig.memoryType) + + val connectionManager = netConfig.nettyConfig match { + case Some(nettyConfig) => new NettyConnectionManager(nettyConfig) + case None => new LocalConnectionManager() + } + + val resultPartitionManager = new ResultPartitionManager() + val taskEventDispatcher = new TaskEventDispatcher() + + val kvStateRegistry = new KvStateRegistry() + + val kvStateServer = netConfig.nettyConfig match { + case Some(nettyConfig) => + + val numNetworkThreads = if (netConfig.queryServerNetworkThreads == 0) { + nettyConfig.getNumberOfSlots + } else { + netConfig.queryServerNetworkThreads + } + + val numQueryThreads = if (netConfig.queryServerQueryThreads == 0) { + nettyConfig.getNumberOfSlots + } else { + netConfig.queryServerQueryThreads + } + + new KvStateServer( + taskManagerAddress.getAddress(), + netConfig.queryServerPort, + numNetworkThreads, + numQueryThreads, + kvStateRegistry, + new DisabledKvStateRequestStats()) + + case None => null + } // we start the network first, to make sure it can allocate its buffers first val network = new NetworkEnvironment( - executionContext, - taskManagerConfig.timeout, - netConfig, - connectionInfo) + networkBufferPool, + connectionManager, + resultPartitionManager, + taskEventDispatcher, + kvStateRegistry, + kvStateServer, + netConfig.ioMode, + netConfig.partitionRequestInitialBackoff, + netConfig.partitinRequestMaxBackoff) + + network.start() + + val connectionInfo = new InstanceConnectionInfo( + taskManagerAddress.getAddress(), + network.getConnectionManager().getDataPort()) // computing the amount of memory to use depends on how much memory is available // it strictly needs to happen AFTER the network stack has been initialized @@ -1991,7 +2064,7 @@ object TaskManager { localTaskManagerCommunication: Boolean) : (TaskManagerConfiguration, NetworkEnvironmentConfiguration, - InstanceConnectionInfo, + InetSocketAddress, MemoryType) = { // ------- read values from the config and check them --------- @@ -2000,16 +2073,13 @@ object TaskManager { // ----> hosts / ports for communication and data exchange val dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, - ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT) match { - case 0 => NetUtils.getAvailablePort() - case x => x - } + ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT) - checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, + checkConfigParameter(dataport >= 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, "Leave config parameter empty or use 0 to let the system choose a port automatically.") val taskManagerAddress = InetAddress.getByName(taskManagerHostname) - val connectionInfo = new InstanceConnectionInfo(taskManagerAddress, dataport) + val taskManagerInetSocketAddress = new InetSocketAddress(taskManagerAddress, dataport) // ----> memory / network stack (shuffles/broadcasts), task slots, temp directories @@ -2076,8 +2146,8 @@ object TaskManager { } else { Some( new NettyConfig( - connectionInfo.address(), - connectionInfo.dataPort(), + taskManagerInetSocketAddress.getAddress(), + taskManagerInetSocketAddress.getPort(), pageSize, slots, configuration) @@ -2206,7 +2276,7 @@ object TaskManager { maxRegistrationPause, refusedRegistrationPause) - (taskManagerConfig, networkConfig, connectionInfo, memType) + (taskManagerConfig, networkConfig, taskManagerInetSocketAddress, memType) } /** diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerLike.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerLike.scala index b41db31933b7b..a6963fe6879af 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerLike.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerLike.scala @@ -108,7 +108,7 @@ trait TestingTaskManagerLike extends FlinkActor { ) case RequestNumActiveConnections => - val numActive = if (network.isAssociated) { + val numActive = if (!network.isShutdown) { network.getConnectionManager.getNumberOfActiveConnections } else { 0 diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java index 4597e3bf2e897..9f39de1c26aac 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java @@ -19,39 +19,34 @@ package org.apache.flink.runtime.io.network; import org.apache.flink.api.common.JobID; -import org.apache.flink.configuration.Configuration; import org.apache.flink.core.memory.MemoryType; import org.apache.flink.runtime.instance.ActorGateway; -import org.apache.flink.runtime.instance.DummyActorGateway; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; -import org.apache.flink.runtime.io.network.buffer.BufferPool; +import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; import org.apache.flink.runtime.io.network.netty.NettyConfig; import org.apache.flink.runtime.io.network.partition.ResultPartition; +import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate; import org.apache.flink.runtime.messages.JobManagerMessages.ScheduleOrUpdateConsumers; +import org.apache.flink.runtime.query.KvStateRegistry; +import org.apache.flink.runtime.taskmanager.ActorGatewayResultPartitionConsumableNotifier; import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; +import org.apache.flink.runtime.taskmanager.JobManagerCommunicationFactory; import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.runtime.testingUtils.TestingUtils; -import org.apache.flink.util.NetUtils; import org.junit.Test; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; import scala.Some; -import scala.Tuple2; import scala.concurrent.duration.FiniteDuration; import scala.concurrent.impl.Promise; -import java.net.InetAddress; import java.util.concurrent.TimeUnit; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyObject; import static org.mockito.Matchers.eq; @@ -61,100 +56,6 @@ import static org.mockito.Mockito.when; public class NetworkEnvironmentTest { - - @Test - public void testAssociateDisassociate() { - final int BUFFER_SIZE = 1024; - final int NUM_BUFFERS = 20; - - final int port; - try { - port = NetUtils.getAvailablePort(); - } - catch (Throwable t) { - // ignore - return; - } - - try { - NettyConfig nettyConf = new NettyConfig(InetAddress.getLocalHost(), port, BUFFER_SIZE, 1, new Configuration()); - NetworkEnvironmentConfiguration config = new NetworkEnvironmentConfiguration( - NUM_BUFFERS, BUFFER_SIZE, MemoryType.HEAP, - IOManager.IOMode.SYNC, 0, 0, 0, new Some<>(nettyConf), - new Tuple2<>(0, 0)); - - NetworkEnvironment env = new NetworkEnvironment( - TestingUtils.defaultExecutionContext(), - new FiniteDuration(30, TimeUnit.SECONDS), - config, - new InstanceConnectionInfo(InetAddress.getLocalHost(), port)); - - assertFalse(env.isShutdown()); - assertFalse(env.isAssociated()); - - // pool must be started already - assertNotNull(env.getNetworkBufferPool()); - assertEquals(NUM_BUFFERS, env.getNetworkBufferPool().getTotalNumberOfMemorySegments()); - - // others components are still shut down - assertNull(env.getConnectionManager()); - assertNull(env.getPartitionConsumableNotifier()); - assertNull(env.getTaskEventDispatcher()); - assertNull(env.getPartitionManager()); - - // associate the environment with some mock actors - env.associateWithTaskManagerAndJobManager( - DummyActorGateway.INSTANCE, - DummyActorGateway.INSTANCE); - - assertNotNull(env.getConnectionManager()); - assertNotNull(env.getPartitionConsumableNotifier()); - assertNotNull(env.getTaskEventDispatcher()); - assertNotNull(env.getPartitionManager()); - - // allocate some buffer pool - BufferPool localPool = env.getNetworkBufferPool().createBufferPool(10, false); - assertNotNull(localPool); - - // disassociate - env.disassociate(); - - assertNull(env.getConnectionManager()); - assertNull(env.getPartitionConsumableNotifier()); - assertNull(env.getTaskEventDispatcher()); - assertNull(env.getPartitionManager()); - - assertNotNull(env.getNetworkBufferPool()); - assertTrue(localPool.isDestroyed()); - - // associate once again - env.associateWithTaskManagerAndJobManager( - DummyActorGateway.INSTANCE, - DummyActorGateway.INSTANCE - ); - - assertNotNull(env.getConnectionManager()); - assertNotNull(env.getPartitionConsumableNotifier()); - assertNotNull(env.getTaskEventDispatcher()); - assertNotNull(env.getPartitionManager()); - - // shutdown for good - env.shutdown(); - - assertTrue(env.isShutdown()); - assertFalse(env.isAssociated()); - assertNull(env.getConnectionManager()); - assertNull(env.getPartitionConsumableNotifier()); - assertNull(env.getTaskEventDispatcher()); - assertNull(env.getPartitionManager()); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - /** * Registers a task with an eager and non-eager partition at the network * environment and verifies that there is exactly on schedule or update @@ -164,45 +65,61 @@ public void testAssociateDisassociate() { @SuppressWarnings("unchecked") public void testEagerlyDeployConsumers() throws Exception { // Mock job manager => expected interactions will be verified - ActorGateway jobManager = mock(ActorGateway.class); + final ActorGateway jobManager = mock(ActorGateway.class); when(jobManager.ask(anyObject(), any(FiniteDuration.class))) .thenReturn(new Promise.DefaultPromise<>().future()); // Network environment setup NetworkEnvironmentConfiguration config = new NetworkEnvironmentConfiguration( - 20, - 1024, - MemoryType.HEAP, - IOManager.IOMode.SYNC, - 0, - 0, - 0, - Some.empty(), - new Tuple2<>(0, 0)); + 20, + 1024, + MemoryType.HEAP, + IOManager.IOMode.SYNC, + 0, + 0, + 0, + Some.empty(), + 0, + 0); NetworkEnvironment env = new NetworkEnvironment( - TestingUtils.defaultExecutionContext(), - new FiniteDuration(30, TimeUnit.SECONDS), - config, - new InstanceConnectionInfo(InetAddress.getLocalHost(), 12232)); - - // Associate the environment with the mock actors - env.associateWithTaskManagerAndJobManager( - jobManager, - DummyActorGateway.INSTANCE); + new NetworkBufferPool(config.numNetworkBuffers(), config.networkBufferSize(), config.memoryType()), + new LocalConnectionManager(), + new ResultPartitionManager(), + new TaskEventDispatcher(), + new KvStateRegistry(), + null, + config.ioMode(), + config.partitionRequestInitialBackoff(), + config.partitinRequestMaxBackoff()); + + env.start(); + + JobManagerCommunicationFactory jobManagerCommunicationFactory = mock(JobManagerCommunicationFactory.class); + + when(jobManagerCommunicationFactory.createResultPartitionConsumableNotifier(any(Task.class))).thenAnswer(new Answer() { + @Override + public ResultPartitionConsumableNotifier answer(InvocationOnMock invocation) throws Throwable { + return new ActorGatewayResultPartitionConsumableNotifier( + TestingUtils.defaultExecutionContext(), + jobManager, + (Task)invocation.getArguments()[0], + new FiniteDuration(30, TimeUnit.SECONDS)); + } + }); // Register mock task JobID jobId = new JobID(); + Task mockTask = mock(Task.class); ResultPartition[] partitions = new ResultPartition[2]; - partitions[0] = createPartition("p1", jobId, true, env); - partitions[1] = createPartition("p2", jobId, false, env); + partitions[0] = createPartition(mockTask, "p1", jobId, true, env, jobManagerCommunicationFactory); + partitions[1] = createPartition(mockTask, "p2", jobId, false, env, jobManagerCommunicationFactory); ResultPartitionWriter[] writers = new ResultPartitionWriter[2]; writers[0] = new ResultPartitionWriter(partitions[0]); writers[1] = new ResultPartitionWriter(partitions[1]); - Task mockTask = mock(Task.class); when(mockTask.getAllInputGates()).thenReturn(new SingleInputGate[0]); when(mockTask.getAllWriters()).thenReturn(writers); when(mockTask.getProducedPartitions()).thenReturn(partitions); @@ -221,10 +138,12 @@ public void testEagerlyDeployConsumers() throws Exception { * Helper to create a mock result partition. */ private static ResultPartition createPartition( - String name, - JobID jobId, - boolean eagerlyDeployConsumers, - NetworkEnvironment env) { + Task owningTask, + String name, + JobID jobId, + boolean eagerlyDeployConsumers, + NetworkEnvironment env, + JobManagerCommunicationFactory jobManagerCommunicationFactory) { return new ResultPartition( name, @@ -233,8 +152,8 @@ private static ResultPartition createPartition( ResultPartitionType.PIPELINED, eagerlyDeployConsumers, 1, - env.getPartitionManager(), - env.getPartitionConsumableNotifier(), + env.getResultPartitionManager(), + jobManagerCommunicationFactory.createResultPartitionConsumableNotifier(owningTask), mock(IOManager.class), env.getDefaultIOMode()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java index 0868398377fb6..8884b295e5977 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java @@ -23,7 +23,6 @@ import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.junit.Test; -import scala.Tuple2; import java.io.IOException; @@ -103,10 +102,11 @@ public void testExponentialNoBackoff() throws Exception { private InputChannel createInputChannel(int initialBackoff, int maxBackoff) { return new MockInputChannel( - mock(SingleInputGate.class), - 0, - new ResultPartitionID(), - new Tuple2(initialBackoff, maxBackoff)); + mock(SingleInputGate.class), + 0, + new ResultPartitionID(), + initialBackoff, + maxBackoff); } // --------------------------------------------------------------------------------------------- @@ -117,9 +117,10 @@ private MockInputChannel( SingleInputGate inputGate, int channelIndex, ResultPartitionID partitionId, - Tuple2 initialAndMaxBackoff) { + int initialBackoff, + int maxBackoff) { - super(inputGate, channelIndex, partitionId, initialAndMaxBackoff, new SimpleCounter()); + super(inputGate, channelIndex, partitionId, initialBackoff, maxBackoff, new SimpleCounter()); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java index f91a4badad8bc..18d90730ce567 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java @@ -267,13 +267,14 @@ private LocalInputChannel createLocalInputChannel( throws IOException, InterruptedException { return new LocalInputChannel( - inputGate, - 0, - new ResultPartitionID(), - partitionManager, - mock(TaskEventDispatcher.class), - initialAndMaxRequestBackoff, - new UnregisteredTaskMetricsGroup.DummyIOMetricGroup()); + inputGate, + 0, + new ResultPartitionID(), + partitionManager, + mock(TaskEventDispatcher.class), + initialAndMaxRequestBackoff._1(), + initialAndMaxRequestBackoff._2(), + new UnregisteredTaskMetricsGroup.DummyIOMetricGroup()); } /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java index 9eb49efa0af6e..9a79ff89d7652 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java @@ -299,12 +299,13 @@ private RemoteInputChannel createRemoteInputChannel( .thenReturn(partitionRequestClient); return new RemoteInputChannel( - inputGate, - 0, - new ResultPartitionID(), - mock(ConnectionID.class), - connectionManager, - initialAndMaxRequestBackoff, - new UnregisteredTaskMetricsGroup.DummyIOMetricGroup()); + inputGate, + 0, + new ResultPartitionID(), + mock(ConnectionID.class), + connectionManager, + initialAndMaxRequestBackoff._1(), + initialAndMaxRequestBackoff._2(), + new UnregisteredTaskMetricsGroup.DummyIOMetricGroup()); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java index 05427a1aa36bc..f55fee521b11b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java @@ -128,7 +128,7 @@ public void testBackwardsEventWithUninitializedChannel() throws Exception { // Unknown ResultPartitionID unknownPartitionId = new ResultPartitionID(new IntermediateResultPartitionID(), new ExecutionAttemptID()); - InputChannel unknown = new UnknownInputChannel(inputGate, 1, unknownPartitionId, partitionManager, taskEventDispatcher, mock(ConnectionManager.class), new Tuple2(0, 0), new UnregisteredTaskMetricsGroup.DummyIOMetricGroup()); + InputChannel unknown = new UnknownInputChannel(inputGate, 1, unknownPartitionId, partitionManager, taskEventDispatcher, mock(ConnectionManager.class), 0, 0, new UnregisteredTaskMetricsGroup.DummyIOMetricGroup()); // Set channels inputGate.setInputChannel(localPartitionId.getPartitionId(), local); @@ -174,13 +174,15 @@ public void testUpdateChannelBeforeRequest() throws Exception { ResultPartitionManager partitionManager = mock(ResultPartitionManager.class); InputChannel unknown = new UnknownInputChannel( - inputGate, - 0, - new ResultPartitionID(), - partitionManager, - new TaskEventDispatcher(), - new LocalConnectionManager(), - new Tuple2(0, 0), new UnregisteredTaskMetricsGroup.DummyIOMetricGroup()); + inputGate, + 0, + new ResultPartitionID(), + partitionManager, + new TaskEventDispatcher(), + new LocalConnectionManager(), + 0, + 0, + new UnregisteredTaskMetricsGroup.DummyIOMetricGroup()); inputGate.setInputChannel(unknown.partitionId.getPartitionId(), unknown); @@ -213,14 +215,15 @@ public void testReleaseWhilePollingChannel() throws Exception { new UnregisteredTaskMetricsGroup.DummyIOMetricGroup()); InputChannel unknown = new UnknownInputChannel( - inputGate, - 0, - new ResultPartitionID(), - new ResultPartitionManager(), - new TaskEventDispatcher(), - new LocalConnectionManager(), - new Tuple2(0, 0), - new UnregisteredTaskMetricsGroup.DummyIOMetricGroup()); + inputGate, + 0, + new ResultPartitionID(), + new ResultPartitionManager(), + new TaskEventDispatcher(), + new LocalConnectionManager(), + 0, + 0, + new UnregisteredTaskMetricsGroup.DummyIOMetricGroup()); inputGate.setInputChannel(unknown.partitionId.getPartitionId(), unknown); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java index ab4ca3b9a5f12..9501c7c58e585 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java @@ -150,12 +150,14 @@ private static Task createTask() throws Exception { ResultPartitionManager partitionManager = mock(ResultPartitionManager.class); ResultPartitionConsumableNotifier consumableNotifier = mock(ResultPartitionConsumableNotifier.class); NetworkEnvironment networkEnvironment = mock(NetworkEnvironment.class); - when(networkEnvironment.getPartitionManager()).thenReturn(partitionManager); - when(networkEnvironment.getPartitionConsumableNotifier()).thenReturn(consumableNotifier); + when(networkEnvironment.getResultPartitionManager()).thenReturn(partitionManager); when(networkEnvironment.getDefaultIOMode()).thenReturn(IOManager.IOMode.SYNC); when(networkEnvironment.createKvStateTaskRegistry(any(JobID.class), any(JobVertexID.class))) .thenReturn(mock(TaskKvStateRegistry.class)); + JobManagerCommunicationFactory jobManagerCommunicationFactory = mock(JobManagerCommunicationFactory.class); + when(jobManagerCommunicationFactory.createResultPartitionConsumableNotifier(any(Task.class))).thenReturn(consumableNotifier); + TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor( new JobID(), "Job Name", new JobVertexID(), new ExecutionAttemptID(), new SerializedValue<>(new ExecutionConfig()), @@ -170,17 +172,18 @@ private static Task createTask() throws Exception { ActorGateway taskManagerGateway = DummyActorGateway.INSTANCE; return new Task(tdd, - mock(MemoryManager.class), - mock(IOManager.class), - networkEnvironment, - mock(BroadcastVariableManager.class), - taskManagerGateway, - DummyActorGateway.INSTANCE, - new FiniteDuration(60, TimeUnit.SECONDS), - libCache, - mock(FileCache.class), - new TaskManagerRuntimeInfo("localhost", new Configuration(), System.getProperty("java.io.tmpdir")), - mock(TaskMetricGroup.class)); + mock(MemoryManager.class), + mock(IOManager.class), + networkEnvironment, + jobManagerCommunicationFactory, + mock(BroadcastVariableManager.class), + taskManagerGateway, + DummyActorGateway.INSTANCE, + new FiniteDuration(60, TimeUnit.SECONDS), + libCache, + mock(FileCache.class), + new TaskManagerRuntimeInfo("localhost", new Configuration(), System.getProperty("java.io.tmpdir")), + mock(TaskMetricGroup.class)); } public static class CheckpointsInOrderInvokable extends AbstractInvokable implements StatefulTask { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java index 147a3e0accef5..3371c49bde702 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java @@ -35,8 +35,12 @@ import org.apache.flink.runtime.instance.InstanceConnectionInfo; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; +import org.apache.flink.runtime.io.network.LocalConnectionManager; import org.apache.flink.runtime.io.network.NetworkEnvironment; +import org.apache.flink.runtime.io.network.TaskEventDispatcher; +import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; import org.apache.flink.runtime.io.network.netty.NettyConfig; +import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; import org.apache.flink.runtime.jobmanager.JobManager; import org.apache.flink.runtime.jobmanager.MemoryArchivist; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; @@ -44,11 +48,11 @@ import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.messages.TaskManagerMessages; +import org.apache.flink.runtime.query.KvStateRegistry; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.util.LeaderRetrievalUtils; import org.junit.Test; import scala.Option; -import scala.Tuple2; import scala.concurrent.duration.FiniteDuration; import java.net.InetAddress; @@ -99,32 +103,40 @@ public void testComponentsStartupShutdown() { final NetworkEnvironmentConfiguration netConf = new NetworkEnvironmentConfiguration( 32, BUFFER_SIZE, MemoryType.HEAP, IOManager.IOMode.SYNC, 0, 0, 0, - Option.empty(), new Tuple2(0, 0)); + Option.empty(), 0, 0); final InstanceConnectionInfo connectionInfo = new InstanceConnectionInfo(InetAddress.getLocalHost(), 10000); final MemoryManager memManager = new MemoryManager(32 * BUFFER_SIZE, 1, BUFFER_SIZE, MemoryType.HEAP, false); final IOManager ioManager = new IOManagerAsync(TMP_DIR); final NetworkEnvironment network = new NetworkEnvironment( - TestingUtils.defaultExecutionContext(), - timeout, - netConf, - connectionInfo); + new NetworkBufferPool(netConf.numNetworkBuffers(), netConf.networkBufferSize(), netConf.memoryType()), + new LocalConnectionManager(), + new ResultPartitionManager(), + new TaskEventDispatcher(), + new KvStateRegistry(), + null, + netConf.ioMode(), + netConf.partitionRequestInitialBackoff(), + netConf.partitinRequestMaxBackoff()); + + network.start(); + final int numberOfSlots = 1; LeaderRetrievalService leaderRetrievalService = new StandaloneLeaderRetrievalService(jobManager.path().toString()); // create the task manager final Props tmProps = Props.create( - TaskManager.class, - tmConfig, - ResourceID.generate(), - connectionInfo, - memManager, - ioManager, - network, - numberOfSlots, - leaderRetrievalService); + TaskManager.class, + tmConfig, + ResourceID.generate(), + connectionInfo, + memManager, + ioManager, + network, + numberOfSlots, + leaderRetrievalService); final ActorRef taskManager = actorSystem.actorOf(tmProps); @@ -142,9 +154,6 @@ protected void run() { }; }}; - // the components should now all be initialized - assertTrue(network.isAssociated()); - // shut down all actors and the actor system // Kill the Task down the JobManager taskManager.tell(Kill.getInstance(), ActorRef.noSender()); @@ -156,7 +165,6 @@ protected void run() { actorSystem = null; // now that the TaskManager is shut down, the components should be shut down as well - assertFalse(network.isAssociated()); assertTrue(network.isShutdown()); assertTrue(ioManager.isProperlyShutDown()); assertTrue(memManager.isShutdown()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerRegistrationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerRegistrationTest.java index e23aba7bfe5b2..53fa7c1975b0c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerRegistrationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerRegistrationTest.java @@ -28,12 +28,10 @@ import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.clusterframework.FlinkResourceManager; import org.apache.flink.runtime.clusterframework.standalone.StandaloneResourceManager; -import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.jobmanager.JobManager; import org.apache.flink.runtime.jobmanager.MemoryArchivist; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService; import org.apache.flink.runtime.messages.JobManagerMessages; import org.apache.flink.runtime.messages.JobManagerMessages.LeaderSessionMessage; @@ -47,15 +45,11 @@ import org.junit.BeforeClass; import org.junit.Test; import scala.Option; -import scala.Some; import scala.concurrent.Await; import scala.concurrent.Future; import scala.concurrent.duration.Deadline; import scala.concurrent.duration.FiniteDuration; -import java.io.IOException; -import java.net.InetAddress; -import java.net.ServerSocket; import java.util.UUID; import java.util.concurrent.TimeUnit; @@ -548,69 +542,6 @@ protected void run() { }}; } - - @Test - public void testStartupWhenNetworkStackFailsToInitialize() { - - ServerSocket blocker = null; - - try { - blocker = new ServerSocket(0, 50, InetAddress.getByName("localhost")); - - final Configuration cfg = new Configuration(); - cfg.setString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, "localhost"); - cfg.setInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, blocker.getLocalPort()); - cfg.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 1); - - new JavaTestKit(actorSystem) {{ - ActorRef taskManager = null; - ActorRef jobManager = null; - ActorRef resourceManager = null; - - try { - // a simple JobManager - jobManager = startJobManager(config); - - resourceManager = startResourceManager(config, jobManager); - - // start a task manager with a configuration that provides a blocked port - taskManager = TaskManager.startTaskManagerComponentsAndActor( - cfg, ResourceID.generate(), actorSystem, "localhost", - NONE_STRING, // no actor name -> random - new Some(new StandaloneLeaderRetrievalService(jobManager.path().toString())), - false, // init network stack !!! - TaskManager.class); - - watch(taskManager); - - expectTerminated(timeout, taskManager); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } finally { - stopActor(taskManager); - stopActor(jobManager); - } - }}; - } - catch (Exception e) { - // does not work, skip test - e.printStackTrace(); - fail(e.getMessage()); - } - finally { - if (blocker != null) { - try { - blocker.close(); - } - catch (IOException e) { - // ignore, best effort - } - } - } - } - @Test public void testCheckForValidRegistrationSessionIDs() { new JavaTestKit(actorSystem) {{ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerStartupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerStartupTest.java index 149df6ecf8c30..686de7617b1b1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerStartupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerStartupTest.java @@ -25,9 +25,10 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.IllegalConfigurationException; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.util.StartupUtils; -import org.junit.Before; import org.junit.Test; +import scala.Option; import java.io.File; import java.io.IOException; @@ -179,4 +180,43 @@ public void testMemoryConfigWrong() { fail(e.getMessage()); } } + + /** + * Tests that the task manager start-up fails if the network stack cannot be initialized. + * @throws Exception + */ + @Test(expected = IOException.class) + public void testStartupWhenNetworkStackFailsToInitialize() throws Exception { + + ServerSocket blocker = null; + + try { + blocker = new ServerSocket(0, 50, InetAddress.getByName("localhost")); + + final Configuration cfg = new Configuration(); + cfg.setString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, "localhost"); + cfg.setInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, blocker.getLocalPort()); + cfg.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 1); + + TaskManager.startTaskManagerComponentsAndActor( + cfg, + ResourceID.generate(), + null, + "localhost", + Option.empty(), + Option.empty(), + false, + TaskManager.class); + } + finally { + if (blocker != null) { + try { + blocker.close(); + } + catch (IOException e) { + // ignore, best effort + } + } + } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java index 54cd7c62326eb..0e53673eab5c9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java @@ -990,11 +990,8 @@ public void testLocalPartitionNotFound() throws Exception { jobManager = new AkkaActorGateway(jm, leaderSessionID); - final int dataPort = NetUtils.getAvailablePort(); final Configuration config = new Configuration(); - config.setInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, dataPort); - taskManager = TestingUtils.createTaskManager( system, jobManager, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java index fea21be79568e..cfa7fb663a2bd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java @@ -66,10 +66,20 @@ public void doMocking(AbstractInvokable taskMock) throws Exception { when(tddMock.getSerializedExecutionConfig()).thenReturn(mock(SerializedValue.class)); when(tddMock.getInvokableClassName()).thenReturn("className"); - task = new Task(tddMock, mock(MemoryManager.class), mock(IOManager.class), mock(NetworkEnvironment.class), - mock(BroadcastVariableManager.class), mock(ActorGateway.class), mock(ActorGateway.class), - mock(FiniteDuration.class), mock(LibraryCacheManager.class), mock(FileCache.class), - mock(TaskManagerRuntimeInfo.class), mock(TaskMetricGroup.class)); + task = new Task( + tddMock, + mock(MemoryManager.class), + mock(IOManager.class), + mock(NetworkEnvironment.class), + mock(JobManagerCommunicationFactory.class), + mock(BroadcastVariableManager.class), + mock(ActorGateway.class), + mock(ActorGateway.class), + mock(FiniteDuration.class), + mock(LibraryCacheManager.class), + mock(FileCache.class), + mock(TaskManagerRuntimeInfo.class), + mock(TaskMetricGroup.class)); Field f = task.getClass().getDeclaredField("invokable"); f.setAccessible(true); f.set(task, taskMock); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java index f145b4866577f..9e8f8f85efe42 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java @@ -244,12 +244,14 @@ public void testExecutionFailsInNetworkRegistration() { ResultPartitionManager partitionManager = mock(ResultPartitionManager.class); ResultPartitionConsumableNotifier consumableNotifier = mock(ResultPartitionConsumableNotifier.class); NetworkEnvironment network = mock(NetworkEnvironment.class); - when(network.getPartitionManager()).thenReturn(partitionManager); - when(network.getPartitionConsumableNotifier()).thenReturn(consumableNotifier); + when(network.getResultPartitionManager()).thenReturn(partitionManager); when(network.getDefaultIOMode()).thenReturn(IOManager.IOMode.SYNC); doThrow(new RuntimeException("buffers")).when(network).registerTask(any(Task.class)); + + JobManagerCommunicationFactory jobManagerCommunicationFactory = mock(JobManagerCommunicationFactory.class); + when(jobManagerCommunicationFactory.createResultPartitionConsumableNotifier(any(Task.class))).thenReturn(consumableNotifier); - Task task = createTask(TestInvokableCorrect.class, libCache, network); + Task task = createTask(TestInvokableCorrect.class, libCache, network, jobManagerCommunicationFactory); task.registerExecutionListener(listenerGateway); @@ -598,18 +600,22 @@ private Task createTask(Class invokable, ResultPartitionManager partitionManager = mock(ResultPartitionManager.class); ResultPartitionConsumableNotifier consumableNotifier = mock(ResultPartitionConsumableNotifier.class); NetworkEnvironment network = mock(NetworkEnvironment.class); - when(network.getPartitionManager()).thenReturn(partitionManager); - when(network.getPartitionConsumableNotifier()).thenReturn(consumableNotifier); + JobManagerCommunicationFactory jobManagerCommunicationFactory = mock(JobManagerCommunicationFactory.class); + when(network.getResultPartitionManager()).thenReturn(partitionManager); when(network.getDefaultIOMode()).thenReturn(IOManager.IOMode.SYNC); when(network.createKvStateTaskRegistry(any(JobID.class), any(JobVertexID.class))) .thenReturn(mock(TaskKvStateRegistry.class)); + + when(jobManagerCommunicationFactory.createResultPartitionConsumableNotifier(any(Task.class))).thenReturn(consumableNotifier); - return createTask(invokable, libCache, network); + return createTask(invokable, libCache, network, jobManagerCommunicationFactory); } - private Task createTask(Class invokable, - LibraryCacheManager libCache, - NetworkEnvironment networkEnvironment) { + private Task createTask( + Class invokable, + LibraryCacheManager libCache, + NetworkEnvironment networkEnvironment, + JobManagerCommunicationFactory jobManagerCommunicationFactory) { TaskDeploymentDescriptor tdd = createTaskDeploymentDescriptor(invokable); @@ -618,6 +624,7 @@ private Task createTask(Class invokable, mock(MemoryManager.class), mock(IOManager.class), networkEnvironment, + jobManagerCommunicationFactory, mock(BroadcastVariableManager.class), taskManagerGateway, jobManagerGateway, diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java index f8b4063280cff..e1c9407bdf970 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java @@ -41,13 +41,12 @@ import org.apache.flink.runtime.query.TaskKvStateRegistry; import org.apache.flink.runtime.state.AbstractCloseableHandle; import org.apache.flink.runtime.state.ChainedStateHandle; +import org.apache.flink.runtime.taskmanager.JobManagerCommunicationFactory; import org.apache.flink.runtime.state.KeyGroupsStateHandle; -import org.apache.flink.runtime.state.RetrievableStateHandle; import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; import org.apache.flink.runtime.util.EnvironmentInformation; -import org.apache.flink.runtime.util.SerializableObject; import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.checkpoint.Checkpointed; import org.apache.flink.streaming.api.functions.source.SourceFunction; @@ -148,19 +147,20 @@ private static Task createTask(TaskDeploymentDescriptor tdd) throws IOException .thenReturn(mock(TaskKvStateRegistry.class)); return new Task( - tdd, - mock(MemoryManager.class), - mock(IOManager.class), - networkEnvironment, - mock(BroadcastVariableManager.class), - mock(ActorGateway.class), - mock(ActorGateway.class), - new FiniteDuration(10, TimeUnit.SECONDS), - new FallbackLibraryCacheManager(), - new FileCache(new Configuration()), - new TaskManagerRuntimeInfo( - "localhost", new Configuration(), EnvironmentInformation.getTemporaryFileDirectory()), - new UnregisteredTaskMetricsGroup()); + tdd, + mock(MemoryManager.class), + mock(IOManager.class), + networkEnvironment, + mock(JobManagerCommunicationFactory.class), + mock(BroadcastVariableManager.class), + mock(ActorGateway.class), + mock(ActorGateway.class), + new FiniteDuration(10, TimeUnit.SECONDS), + new FallbackLibraryCacheManager(), + new FileCache(new Configuration()), + new TaskManagerRuntimeInfo( + "localhost", new Configuration(), EnvironmentInformation.getTemporaryFileDirectory()), + new UnregisteredTaskMetricsGroup()); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java index 408b5b14436c5..0a9d2faa900f9 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java @@ -45,6 +45,7 @@ import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.messages.TaskMessages; import org.apache.flink.runtime.query.TaskKvStateRegistry; +import org.apache.flink.runtime.taskmanager.JobManagerCommunicationFactory; import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; import org.apache.flink.streaming.api.TimeCharacteristic; @@ -81,9 +82,6 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; public class StreamTaskTest { @@ -228,12 +226,14 @@ private Task createTask(Class invokable, StreamConf ResultPartitionManager partitionManager = mock(ResultPartitionManager.class); ResultPartitionConsumableNotifier consumableNotifier = mock(ResultPartitionConsumableNotifier.class); NetworkEnvironment network = mock(NetworkEnvironment.class); - when(network.getPartitionManager()).thenReturn(partitionManager); - when(network.getPartitionConsumableNotifier()).thenReturn(consumableNotifier); + when(network.getResultPartitionManager()).thenReturn(partitionManager); when(network.getDefaultIOMode()).thenReturn(IOManager.IOMode.SYNC); when(network.createKvStateTaskRegistry(any(JobID.class), any(JobVertexID.class))) .thenReturn(mock(TaskKvStateRegistry.class)); + JobManagerCommunicationFactory jobManagerCommunicationFactory = mock(JobManagerCommunicationFactory.class); + when(jobManagerCommunicationFactory.createResultPartitionConsumableNotifier(any(Task.class))).thenReturn(consumableNotifier); + TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor( new JobID(), "Job Name", new JobVertexID(), new ExecutionAttemptID(), new SerializedValue<>(new ExecutionConfig()), @@ -248,18 +248,19 @@ private Task createTask(Class invokable, StreamConf 0); return new Task( - tdd, - mock(MemoryManager.class), - mock(IOManager.class), - network, - mock(BroadcastVariableManager.class), - new DummyGateway(), - new DummyGateway(), - new FiniteDuration(60, TimeUnit.SECONDS), - libCache, - mock(FileCache.class), - new TaskManagerRuntimeInfo("localhost", new Configuration(), System.getProperty("java.io.tmpdir")), - mock(TaskMetricGroup.class)); + tdd, + mock(MemoryManager.class), + mock(IOManager.class), + network, + jobManagerCommunicationFactory, + mock(BroadcastVariableManager.class), + new DummyGateway(), + new DummyGateway(), + new FiniteDuration(60, TimeUnit.SECONDS), + libCache, + mock(FileCache.class), + new TaskManagerRuntimeInfo("localhost", new Configuration(), System.getProperty("java.io.tmpdir")), + mock(TaskMetricGroup.class)); } // ------------------------------------------------------------------------ diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java index 82dbd1f534618..8915bfffd3354 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java @@ -233,13 +233,19 @@ public void testSavepointRescalingFailureWithNonPartitionedState() throws Except cluster.submitJobDetached(jobGraph); - Future allTasksRunning = jobManager.ask(new TestingJobManagerMessages.WaitForAllVerticesToBeRunning(jobID), deadline.timeLeft()); + Object savepointResponse = null; - Await.ready(allTasksRunning, deadline.timeLeft()); + // we might be too early for taking a savepoint if the operators have not been started yet + while (deadline.hasTimeLeft()) { - Future savepointPathFuture = jobManager.ask(new JobManagerMessages.TriggerSavepoint(jobID), deadline.timeLeft()); + Future savepointPathFuture = jobManager.ask(new JobManagerMessages.TriggerSavepoint(jobID), deadline.timeLeft()); + + savepointResponse = Await.result(savepointPathFuture, deadline.timeLeft()); - Object savepointResponse = Await.result(savepointPathFuture, deadline.timeLeft()); + if (savepointResponse instanceof JobManagerMessages.TriggerSavepointSuccess) { + break; + } + } assertTrue(savepointResponse instanceof JobManagerMessages.TriggerSavepointSuccess); diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala index 8b39f522929a3..107801dd6c2df 100644 --- a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala +++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala @@ -18,17 +18,13 @@ package org.apache.flink.yarn -import org.apache.flink.runtime.clusterframework.messages.StopCluster import org.apache.flink.runtime.clusterframework.types.ResourceID import org.apache.flink.runtime.instance.InstanceConnectionInfo import org.apache.flink.runtime.io.disk.iomanager.IOManager import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService import org.apache.flink.runtime.memory.MemoryManager -import org.apache.flink.runtime.taskmanager.{TaskManagerConfiguration, TaskManager} -import org.apache.flink.runtime.util.ProcessShutDownThread - -import scala.concurrent.duration._ +import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerConfiguration} /** An extension of the TaskManager that listens for additional YARN related * messages. From d7ec1e04e85563742e658a11986932350d30069e Mon Sep 17 00:00:00 2001 From: twalthr Date: Fri, 2 Sep 2016 16:23:04 +0200 Subject: [PATCH 012/299] [FLINK-3497] [table] Add IS (NOT) TRUE/IS (NOT) FALSE functions --- docs/dev/table_api.md | 94 ++++++++++++++++++- .../flink/api/scala/table/expressionDsl.scala | 10 ++ .../api/table/codegen/CodeGenerator.scala | 20 ++++ .../table/codegen/calls/ScalarOperators.scala | 32 +++++++ .../flink/api/table/codegen/generated.scala | 6 ++ .../table/expressions/ExpressionParser.scala | 33 +++---- .../api/table/expressions/comparison.scala | 20 ++++ .../api/table/validate/FunctionCatalog.scala | 6 ++ .../expressions/ScalarFunctionsTest.scala | 57 ++++++++++- 9 files changed, 253 insertions(+), 25 deletions(-) diff --git a/docs/dev/table_api.md b/docs/dev/table_api.md index 543945cf865e9..68a2b956858af 100644 --- a/docs/dev/table_api.md +++ b/docs/dev/table_api.md @@ -963,7 +963,7 @@ unary = [ "!" | "-" ] , composite ; composite = suffixed | atom ; -suffixed = interval | cast | as | aggregation | nullCheck | if | functionCall ; +suffixed = interval | cast | as | aggregation | if | functionCall ; interval = composite , "." , ("year" | "month" | "day" | "hour" | "minute" | "second" | "milli") ; @@ -975,11 +975,9 @@ as = composite , ".as(" , fieldReference , ")" ; aggregation = composite , ( ".sum" | ".min" | ".max" | ".count" | ".avg" ) , [ "()" ] ; -nullCheck = composite , ( ".isNull" | ".isNotNull" ) , [ "()" ] ; - if = composite , ".?(" , expression , "," , expression , ")" ; -functionCall = composite , "." , functionIdentifier , "(" , [ expression , { "," , expression } ] , ")" ; +functionCall = composite , "." , functionIdentifier , [ "(" , [ expression , { "," , expression } ] , ")" ] ; atom = ( "(" , expression , ")" ) | literal | nullLiteral | fieldReference ; @@ -1231,6 +1229,50 @@ Both the Table API and SQL come with a set of built-in scalar functions for data + + + {% highlight java %} +ANY.isNull +{% endhighlight %} + + +

Returns true if the given expression is null.

+ + + + + + {% highlight java %} +ANY.isNotNull +{% endhighlight %} + + +

Returns true if the given expression is not null.

+ + + + + + {% highlight java %} +BOOLEAN.isTrue +{% endhighlight %} + + +

Returns true if the given boolean expression is true. False otherwise (for null and false).

+ + + + + + {% highlight java %} +BOOLEAN.isFalse +{% endhighlight %} + + +

Returns true if given boolean expression is false. False otherwise (for null and true).

+ + + {% highlight java %} @@ -1495,6 +1537,50 @@ TIMEPOINT.ceil(TIMEINTERVALUNIT) + + + {% highlight scala %} +ANY.isNull +{% endhighlight %} + + +

Returns true if the given expression is null.

+ + + + + + {% highlight scala %} +ANY.isNotNull +{% endhighlight %} + + +

Returns true if the given expression is not null.

+ + + + + + {% highlight scala %} +BOOLEAN.isTrue +{% endhighlight %} + + +

Returns true if the given boolean expression is true. False otherwise (for null and false).

+ + + + + + {% highlight scala %} +BOOLEAN.isFalse +{% endhighlight %} + + +

Returns true if given boolean expression is false. False otherwise (for null and true).

+ + + {% highlight scala %} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/expressionDsl.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/expressionDsl.scala index b14ca8801cb1b..9bfe6c39d6bee 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/expressionDsl.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/expressionDsl.scala @@ -63,6 +63,16 @@ trait ImplicitExpressionOperations { def isNull = IsNull(expr) def isNotNull = IsNotNull(expr) + /** + * Returns true if given boolean expression is true. False otherwise (for null and false). + */ + def isTrue = IsTrue(expr) + + /** + * Returns true if given boolean expression is false. False otherwise (for null and true). + */ + def isFalse = IsFalse(expr) + def + (other: Expression) = Plus(expr, other) def - (other: Expression) = Minus(expr, other) def / (other: Expression) = Div(expr, other) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala index 4a3865f2b25fa..6463ff99331bc 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala @@ -797,6 +797,26 @@ class CodeGenerator( case CASE => generateIfElse(nullCheck, operands, resultType) + case IS_TRUE => + val operand = operands.head + requireBoolean(operand) + generateIsTrue(operand) + + case IS_NOT_TRUE => + val operand = operands.head + requireBoolean(operand) + generateIsNotTrue(operand) + + case IS_FALSE => + val operand = operands.head + requireBoolean(operand) + generateIsFalse(operand) + + case IS_NOT_FALSE => + val operand = operands.head + requireBoolean(operand) + generateIsNotFalse(operand) + // casting case CAST | REINTERPRET => val operand = operands.head diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/ScalarOperators.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/ScalarOperators.scala index afe69edbb44c8..094a22453b5e0 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/ScalarOperators.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/ScalarOperators.scala @@ -384,6 +384,38 @@ object ScalarOperators { } } + def generateIsTrue(operand: GeneratedExpression): GeneratedExpression = { + GeneratedExpression( + operand.resultTerm, // unknown is always false by default + GeneratedExpression.NEVER_NULL, + operand.code, + BOOLEAN_TYPE_INFO) + } + + def generateIsNotTrue(operand: GeneratedExpression): GeneratedExpression = { + GeneratedExpression( + s"(!${operand.resultTerm})", // unknown is always false by default + GeneratedExpression.NEVER_NULL, + operand.code, + BOOLEAN_TYPE_INFO) + } + + def generateIsFalse(operand: GeneratedExpression): GeneratedExpression = { + GeneratedExpression( + s"(!${operand.resultTerm} && !${operand.nullTerm})", + GeneratedExpression.NEVER_NULL, + operand.code, + BOOLEAN_TYPE_INFO) + } + + def generateIsNotFalse(operand: GeneratedExpression): GeneratedExpression = { + GeneratedExpression( + s"(${operand.resultTerm} || ${operand.nullTerm})", + GeneratedExpression.NEVER_NULL, + operand.code, + BOOLEAN_TYPE_INFO) + } + def generateCast( nullCheck: Boolean, operand: GeneratedExpression, diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/generated.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/generated.scala index 26c669618c0f6..bb52ad893b955 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/generated.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/generated.scala @@ -34,4 +34,10 @@ case class GeneratedExpression( code: String, resultType: TypeInformation[_]) +object GeneratedExpression { + val ALWAYS_NULL = "true" + val NEVER_NULL = "false" + val NO_CODE = "" +} + case class GeneratedFunction[T](name: String, returnType: TypeInformation[Any], code: String) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ExpressionParser.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ExpressionParser.scala index cb925734821e9..ae027e96b2fb8 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ExpressionParser.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ExpressionParser.scala @@ -54,8 +54,6 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers { lazy val MIN: Keyword = Keyword("min") lazy val MAX: Keyword = Keyword("max") lazy val SUM: Keyword = Keyword("sum") - lazy val IS_NULL: Keyword = Keyword("isNull") - lazy val IS_NOT_NULL: Keyword = Keyword("isNotNull") lazy val CAST: Keyword = Keyword("cast") lazy val NULL: Keyword = Keyword("Null") lazy val IF: Keyword = Keyword("?") @@ -79,7 +77,7 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers { def functionIdent: ExpressionParser.Parser[String] = not(AS) ~ not(COUNT) ~ not(AVG) ~ not(MIN) ~ not(MAX) ~ - not(SUM) ~ not(IS_NULL) ~ not(IS_NOT_NULL) ~ not(CAST) ~ not(NULL) ~ + not(SUM) ~ not(CAST) ~ not(NULL) ~ not(IF) ~> super.ident // symbols @@ -169,12 +167,6 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers { // suffix operators - lazy val suffixIsNull: PackratParser[Expression] = - composite <~ "." ~ IS_NULL ~ opt("()") ^^ { e => IsNull(e) } - - lazy val suffixIsNotNull: PackratParser[Expression] = - composite <~ "." ~ IS_NOT_NULL ~ opt("()") ^^ { e => IsNotNull(e) } - lazy val suffixSum: PackratParser[Expression] = composite <~ "." ~ SUM ~ opt("()") ^^ { e => Sum(e) } @@ -230,6 +222,10 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers { case operand ~ _ ~ name ~ _ ~ args ~ _ => Call(name.toUpperCase, operand :: args) } + lazy val suffixFunctionCallOneArg = composite ~ "." ~ functionIdent ^^ { + case operand ~ _ ~ name => Call(name.toUpperCase, Seq(operand)) + } + lazy val suffixAsc : PackratParser[Expression] = atom <~ "." ~ ASC ~ opt("()") ^^ { e => Asc(e) } @@ -264,20 +260,14 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers { } lazy val suffixed: PackratParser[Expression] = - suffixTimeInterval | suffixIsNull | suffixIsNotNull | suffixSum | suffixMin | suffixMax | + suffixTimeInterval | suffixSum | suffixMin | suffixMax | suffixCount | suffixAvg | suffixCast | suffixAs | suffixTrim | suffixTrimWithoutArgs | suffixIf | suffixAsc | suffixDesc | suffixToDate | suffixToTimestamp | suffixToTime | suffixExtract | suffixFloor | suffixCeil | - suffixFunctionCall // function call must always be at the end + suffixFunctionCall | suffixFunctionCallOneArg // function call must always be at the end // prefix operators - lazy val prefixIsNull: PackratParser[Expression] = - IS_NULL ~ "(" ~> expression <~ ")" ^^ { e => IsNull(e) } - - lazy val prefixIsNotNull: PackratParser[Expression] = - IS_NOT_NULL ~ "(" ~> expression <~ ")" ^^ { e => IsNotNull(e) } - lazy val prefixSum: PackratParser[Expression] = SUM ~ "(" ~> expression <~ ")" ^^ { e => Sum(e) } @@ -312,6 +302,10 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers { case name ~ _ ~ args ~ _ => Call(name.toUpperCase, args) } + lazy val prefixFunctionCallOneArg = functionIdent ~ "(" ~ expression ~ ")" ^^ { + case name ~ _ ~ arg ~ _ => Call(name.toUpperCase, Seq(arg)) + } + lazy val prefixTrim = TRIM ~ "(" ~ trimMode ~ "," ~ expression ~ "," ~ expression ~ ")" ^^ { case _ ~ _ ~ mode ~ _ ~ trimCharacter ~ _ ~ operand ~ _ => Trim(mode, trimCharacter, operand) } @@ -333,9 +327,10 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers { } lazy val prefixed: PackratParser[Expression] = - prefixIsNull | prefixIsNotNull | prefixSum | prefixMin | prefixMax | prefixCount | prefixAvg | + prefixSum | prefixMin | prefixMax | prefixCount | prefixAvg | prefixCast | prefixAs | prefixTrim | prefixTrimWithoutArgs | prefixIf | prefixExtract | - prefixFloor | prefixCeil | prefixFunctionCall // function call must always be at the end + prefixFloor | prefixCeil | + prefixFunctionCall | prefixFunctionCallOneArg // function call must always be at the end // suffix/prefix composite diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/comparison.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/comparison.scala index 0acfbf1519b34..ad0167457e2ac 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/comparison.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/comparison.scala @@ -121,3 +121,23 @@ case class IsNotNull(child: Expression) extends UnaryExpression { override private[flink] def resultType = BOOLEAN_TYPE_INFO } + +case class IsTrue(child: Expression) extends UnaryExpression { + override def toString = s"($child).isTrue" + + override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = { + relBuilder.call(SqlStdOperatorTable.IS_TRUE, child.toRexNode) + } + + override private[flink] def resultType = BOOLEAN_TYPE_INFO +} + +case class IsFalse(child: Expression) extends UnaryExpression { + override def toString = s"($child).isFalse" + + override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = { + relBuilder.call(SqlStdOperatorTable.IS_FALSE, child.toRexNode) + } + + override private[flink] def resultType = BOOLEAN_TYPE_INFO +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/validate/FunctionCatalog.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/validate/FunctionCatalog.scala index fb38dde3b3e6a..b9a3f71c92725 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/validate/FunctionCatalog.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/validate/FunctionCatalog.scala @@ -117,6 +117,12 @@ class FunctionCatalog { object FunctionCatalog { val buildInFunctions: Map[String, Class[_]] = Map( + // logic + "isNull" -> classOf[IsNull], + "isNotNull" -> classOf[IsNotNull], + "isTrue" -> classOf[IsTrue], + "isFalse" -> classOf[IsFalse], + // aggregate functions "avg" -> classOf[Avg], "count" -> classOf[Count], diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarFunctionsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarFunctionsTest.scala index 7162a04c81292..7ab0c7df6ae11 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarFunctionsTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarFunctionsTest.scala @@ -702,10 +702,61 @@ class ScalarFunctionsTest extends ExpressionTestBase { "1996-11-01") } + @Test + def testIsTrueIsFalse(): Unit = { + testAllApis( + 'f1.isTrue, + "f1.isTrue", + "f1 IS TRUE", + "true") + + testAllApis( + 'f21.isTrue, + "f21.isTrue", + "f21 IS TRUE", + "false") + + testAllApis( + false.isFalse, + "false.isFalse", + "FALSE IS FALSE", + "true") + + testAllApis( + 'f21.isFalse, + "f21.isFalse", + "f21 IS FALSE", + "false") + + testAllApis( + !'f1.isTrue, + "!f1.isTrue", + "f1 IS NOT TRUE", + "false") + + testAllApis( + !'f21.isTrue, + "!f21.isTrue", + "f21 IS NOT TRUE", + "true") + + testAllApis( + !false.isFalse, + "!false.isFalse", + "FALSE IS NOT FALSE", + "false") + + testAllApis( + !'f21.isFalse, + "!f21.isFalse", + "f21 IS NOT FALSE", + "true") + } + // ---------------------------------------------------------------------------------------------- def testData = { - val testData = new Row(21) + val testData = new Row(22) testData.setField(0, "This is a test String.") testData.setField(1, true) testData.setField(2, 42.toByte) @@ -727,6 +778,7 @@ class ScalarFunctionsTest extends ExpressionTestBase { testData.setField(18, Timestamp.valueOf("1996-11-10 06:55:44.333")) testData.setField(19, 1467012213000L) // +16979 07:23:33.000 testData.setField(20, 25) // +2-01 + testData.setField(21, null) testData } @@ -752,6 +804,7 @@ class ScalarFunctionsTest extends ExpressionTestBase { Types.TIME, Types.TIMESTAMP, Types.INTERVAL_MILLIS, - Types.INTERVAL_MONTHS)).asInstanceOf[TypeInformation[Any]] + Types.INTERVAL_MONTHS, + Types.BOOLEAN)).asInstanceOf[TypeInformation[Any]] } } From 93a6a243e0cac55084c5db88c8643d94724f60eb Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Fri, 2 Sep 2016 17:04:51 +0200 Subject: [PATCH 013/299] [FLINK-4570] disable Scalastyle for flink-mesos Utils file The version change didn't cause the Scalastyle errors. Seems like the only viable solution to prevent random failures of the Scalastyle plugin is to disable Scalastyle checks for the affected source file. --- flink-mesos/src/test/scala/org/apache/flink/mesos/Utils.scala | 3 +++ pom.xml | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/flink-mesos/src/test/scala/org/apache/flink/mesos/Utils.scala b/flink-mesos/src/test/scala/org/apache/flink/mesos/Utils.scala index 80186cf54b45b..6f27795fbfb1e 100644 --- a/flink-mesos/src/test/scala/org/apache/flink/mesos/Utils.scala +++ b/flink-mesos/src/test/scala/org/apache/flink/mesos/Utils.scala @@ -16,6 +16,8 @@ * limitations under the License. */ +// disable Scalastyle for now to prevent random failures reported in FLINK-4570 +// scalastyle:off package org.apache.flink.mesos import java.util.concurrent.atomic.AtomicLong @@ -52,3 +54,4 @@ object TestFSMUtils { new TestFSMRef(system, Props(factory), supervisor, TestFSMUtils.randomName) } } +// scalastyle:on diff --git a/pom.xml b/pom.xml index 28ff9f1f7b4cc..d92944fa87f83 100644 --- a/pom.xml +++ b/pom.xml @@ -1157,7 +1157,7 @@ under the License. org.scalastyle scalastyle-maven-plugin - 0.5.0 + 0.8.0 From 4bb20467c13a023dcf31a3cf93f68557b8cd161e Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Fri, 2 Sep 2016 11:38:53 +0200 Subject: [PATCH 014/299] [FLINK-4567] [runtime] Enhance SerializedThrowable to properly mimic Exception causes --- .../runtime/util/SerializedThrowable.java | 88 +++++++++---------- .../runtime/util/SerializedThrowableTest.java | 40 ++++++++- 2 files changed, 83 insertions(+), 45 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedThrowable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedThrowable.java index a7739ef321a54..4dea59c3737e0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedThrowable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedThrowable.java @@ -21,18 +21,19 @@ import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.InstantiationUtil; -import java.io.IOException; import java.io.PrintStream; import java.io.PrintWriter; import java.io.Serializable; import java.lang.ref.WeakReference; +import java.util.HashSet; +import java.util.Set; /** * Utility class for dealing with user-defined Throwable types that are serialized (for * example during RPC/Actor communication), but cannot be resolved with the default * class loader. - *

- * This exception mimics the original exception with respect to message and stack trace, + * + *

This exception mimics the original exception with respect to message and stack trace, * and contains the original exception in serialized form. The original exception * can be re-obtained by supplying the appropriate class loader. */ @@ -49,10 +50,6 @@ public class SerializedThrowable extends Exception implements Serializable { /** The original stack trace, to be printed */ private final String fullStingifiedStackTrace; - /** A guaranteed serializable placeholder exception that will be used as - * cause and to capture the original stack trace */ - private final Exception placeholder; - /** The original exception, not transported via serialization, * because the class may not be part of the system class loader. * In addition, we make sure our cached references to not prevent @@ -66,33 +63,43 @@ public class SerializedThrowable extends Exception implements Serializable { * @param exception The exception to serialize. */ public SerializedThrowable(Throwable exception) { + this(exception, new HashSet()); + } + + private SerializedThrowable(Throwable exception, Set alreadySeen) { super(getMessageOrError(exception)); if (!(exception instanceof SerializedThrowable)) { - this.cachedException = new WeakReference(exception); - - this.originalErrorClassName = exception.getClass().getName(); - this.fullStingifiedStackTrace = ExceptionUtils.stringifyException(exception); - this.placeholder = new Exception( - "Serialized representation of " + originalErrorClassName + ": " + getMessage()); - this.placeholder.setStackTrace(exception.getStackTrace()); - initCause(this.placeholder); - + // serialize and memoize the original message byte[] serialized; try { serialized = InstantiationUtil.serializeObject(exception); } catch (Throwable t) { - // could not serialize exception. send the stringified version instead - try { - serialized = InstantiationUtil.serializeObject(placeholder); - } - catch (IOException e) { - // this should really never happen, as we only serialize a a standard exception - throw new RuntimeException(e.getMessage(), e); - } + serialized = null; } this.serializedException = serialized; + this.cachedException = new WeakReference(exception); + + // record the original exception's properties (name, stack prints) + this.originalErrorClassName = exception.getClass().getName(); + this.fullStingifiedStackTrace = ExceptionUtils.stringifyException(exception); + + // mimic the original exception's stack trace + setStackTrace(exception.getStackTrace()); + + // mimic the original exception's cause + if (exception.getCause() == null) { + initCause(null); + } + else { + // exception causes may by cyclic, so we truncate the cycle when we find it + if (alreadySeen.add(exception)) { + // we are not in a cycle, yet + initCause(new SerializedThrowable(exception.getCause(), alreadySeen)); + } + } + } else { // copy from that serialized throwable @@ -100,38 +107,36 @@ public SerializedThrowable(Throwable exception) { this.serializedException = other.serializedException; this.originalErrorClassName = other.originalErrorClassName; this.fullStingifiedStackTrace = other.fullStingifiedStackTrace; - this.placeholder = other.placeholder; this.cachedException = other.cachedException; } } public Throwable deserializeError(ClassLoader classloader) { + if (serializedException == null) { + // failed to serialize the original exception + // return this SerializedThrowable as a stand in + return this; + } + Throwable cached = cachedException == null ? null : cachedException.get(); if (cached == null) { try { cached = InstantiationUtil.deserializeObject(serializedException, classloader); cachedException = new WeakReference(cached); } - catch (Exception e) { - return placeholder; + catch (Throwable t) { + // something went wrong + // return this SerializedThrowable as a stand in + return this; } } return cached; } - - public String getStrigifiedStackTrace() { - return fullStingifiedStackTrace; - } - + // ------------------------------------------------------------------------ // Override the behavior of Throwable // ------------------------------------------------------------------------ - @Override - public Throwable getCause() { - return placeholder; - } - @Override public void printStackTrace(PrintStream s) { s.print(fullStingifiedStackTrace); @@ -150,15 +155,10 @@ public String toString() { return (message != null) ? (originalErrorClassName + ": " + message) : originalErrorClassName; } - @Override - public StackTraceElement[] getStackTrace() { - return placeholder.getStackTrace(); - } - // ------------------------------------------------------------------------ // Static utilities // ------------------------------------------------------------------------ - + public static Throwable get(Throwable serThrowable, ClassLoader loader) { if (serThrowable instanceof SerializedThrowable) { return ((SerializedThrowable)serThrowable).deserializeError(loader); @@ -166,7 +166,7 @@ public static Throwable get(Throwable serThrowable, ClassLoader loader) { return serThrowable; } } - + private static String getMessageOrError(Throwable error) { try { return error.getMessage(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/SerializedThrowableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/SerializedThrowableTest.java index 50efd5247dbf5..4d57892869373 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/SerializedThrowableTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/SerializedThrowableTest.java @@ -131,10 +131,48 @@ public void testSerialization() { // deserialize the proper exception Throwable deserialized = copy.deserializeError(loader); assertEquals(clazz, deserialized.getClass()); + + // deserialization with the wrong classloader does not lead to a failure + Throwable wronglyDeserialized = copy.deserializeError(getClass().getClassLoader()); + assertEquals(ExceptionUtils.stringifyException(userException), + ExceptionUtils.stringifyException(wronglyDeserialized)); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); } - } + } + + @Test + public void testCauseChaining() { + Exception cause2 = new Exception("level2"); + Exception cause1 = new Exception("level1", cause2); + Exception root = new Exception("level0", cause1); + + SerializedThrowable st = new SerializedThrowable(root); + + assertEquals("level0", st.getMessage()); + + assertNotNull(st.getCause()); + assertEquals("level1", st.getCause().getMessage()); + + assertNotNull(st.getCause().getCause()); + assertEquals("level2", st.getCause().getCause().getMessage()); + } + + @Test + public void testCyclicCauseChaining() { + Exception cause3 = new Exception("level3"); + Exception cause2 = new Exception("level2", cause3); + Exception cause1 = new Exception("level1", cause2); + Exception root = new Exception("level0", cause1); + + // introduce a cyclic reference + cause3.initCause(cause1); + + SerializedThrowable st = new SerializedThrowable(root); + + assertArrayEquals(root.getStackTrace(), st.getStackTrace()); + assertEquals(ExceptionUtils.stringifyException(root), ExceptionUtils.stringifyException(st)); + } } From c7dd022c5f7c010a28fbfe5d7563663b4ede74e7 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Fri, 2 Sep 2016 11:45:25 +0200 Subject: [PATCH 015/299] [FLINK-4566] [network runtime] Properly preserve exception causes for ProducerFailedException --- .../partition/ProducerFailedException.java | 19 +++++-------------- .../ProducerFailedExceptionTest.java | 12 ++++++------ 2 files changed, 11 insertions(+), 20 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ProducerFailedException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ProducerFailedException.java index 2b2acabf278db..934234da897ce 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ProducerFailedException.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ProducerFailedException.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.io.network.partition; import org.apache.flink.runtime.execution.CancelTaskException; -import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.runtime.util.SerializedThrowable; /** * Network-stack level Exception to notify remote receiver about a failed @@ -29,23 +29,14 @@ public class ProducerFailedException extends CancelTaskException { private static final long serialVersionUID = -1555492656299526395L; - private final String causeAsString; - /** * The cause of the producer failure. * - * Note: The cause will be stringified, because it might be an instance of - * a user level Exception, which can not be deserialized by the remote - * receiver's system class loader. + *

The cause will be stored as a {@link SerializedThrowable}, because it might + * be an instance of a user level Exception, which may not be possible to deserialize + * by the remote receiver's system class loader. */ public ProducerFailedException(Throwable cause) { - this.causeAsString = cause != null ? ExceptionUtils.stringifyException(cause) : null; - } - - /** - * Returns the stringified cause of the producer failure. - */ - public String getCauseAsString() { - return causeAsString; + super(new SerializedThrowable(cause)); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ProducerFailedExceptionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ProducerFailedExceptionTest.java index 042c136dfde9a..ca2de0ca83e40 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ProducerFailedExceptionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ProducerFailedExceptionTest.java @@ -19,27 +19,27 @@ package org.apache.flink.runtime.io.network.partition; import org.apache.flink.runtime.execution.CancelTaskException; +import org.apache.flink.runtime.util.SerializedThrowable; + import org.junit.Test; import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; public class ProducerFailedExceptionTest { @Test public void testInstanceOfCancelTaskException() throws Exception { - ProducerFailedException e = new ProducerFailedException(new Exception()); - assertTrue(e instanceof CancelTaskException); + assertTrue(CancelTaskException.class.isAssignableFrom(ProducerFailedException.class)); } @Test - public void testCauseIsStringified() throws Exception { + public void testCauseIsSerialized() throws Exception { // Tests that the cause is stringified, because it might be an instance // of a user level Exception, which can not be deserialized by the // remote receiver's system class loader. ProducerFailedException e = new ProducerFailedException(new Exception()); - assertNull(e.getCause()); - assertNotNull(e.getCauseAsString()); + assertNotNull(e.getCause()); + assertTrue(e.getCause() instanceof SerializedThrowable); } } From 0c31623bc42a1db4ed886a6ceef027030bfb13f7 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Mon, 29 Aug 2016 16:58:31 +0200 Subject: [PATCH 016/299] [FLINK-4490] [distributed coordination] (part 1) Change InstanceConnectionInfo to TaskManagerLocation This adds the ResourceId to the TaskManagerLocation --- .../clusterframework/MesosTaskManager.scala | 7 +- .../handlers/JobExceptionsHandler.java | 4 +- .../handlers/JobVertexDetailsHandler.java | 4 +- .../JobVertexTaskManagersHandler.java | 4 +- ...SubtaskExecutionAttemptDetailsHandler.java | 4 +- .../SubtasksAllAccumulatorsHandler.java | 4 +- .../handlers/SubtasksTimesHandler.java | 4 +- ...rtialInputChannelDeploymentDescriptor.java | 10 +- .../runtime/executiongraph/Execution.java | 8 +- .../executiongraph/ExecutionVertex.java | 4 +- .../flink/runtime/instance/Instance.java | 7 +- .../runtime/instance/InstanceManager.java | 3 +- .../runtime/io/network/ConnectionID.java | 4 +- .../TaskManagerLocation.java} | 192 +++++++++--------- .../messages/RegistrationMessages.scala | 12 +- .../runtime/taskmanager/TaskManager.scala | 15 +- .../testingUtils/TestingTaskManager.scala | 33 ++- .../ExecutionGraphMetricsTest.java | 8 +- .../ExecutionGraphTestUtils.java | 7 +- .../TerminalStateDeadlockTest.java | 7 +- .../VertexLocationConstraintTest.java | 6 +- .../runtime/instance/InstanceManagerTest.java | 64 +++--- .../flink/runtime/instance/InstanceTest.java | 16 +- .../runtime/instance/SimpleSlotTest.java | 6 +- .../scheduler/SchedulerTestUtils.java | 25 +-- .../ResourceManagerITCase.java | 5 +- ...kManagerComponentsStartupShutdownTest.java | 12 +- .../TaskManagerLocationTest.java} | 49 ++--- .../JobManagerRegistrationTest.scala | 14 +- .../flink/yarn/TestingYarnTaskManager.scala | 20 +- .../apache/flink/yarn/YarnTaskManager.scala | 7 +- 31 files changed, 276 insertions(+), 289 deletions(-) rename flink-runtime/src/main/java/org/apache/flink/runtime/{instance/InstanceConnectionInfo.java => taskmanager/TaskManagerLocation.java} (61%) rename flink-runtime/src/test/java/org/apache/flink/runtime/{instance/InstanceConnectionInfoTest.java => taskmanager/TaskManagerLocationTest.java} (75%) diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala index d8b67755856e7..19b0c625de10e 100644 --- a/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala +++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala @@ -19,12 +19,11 @@ package org.apache.flink.mesos.runtime.clusterframework import org.apache.flink.runtime.clusterframework.types.ResourceID -import org.apache.flink.runtime.instance.InstanceConnectionInfo import org.apache.flink.runtime.io.disk.iomanager.IOManager import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService import org.apache.flink.runtime.memory.MemoryManager -import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerConfiguration} +import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerConfiguration, TaskManagerLocation} /** An extension of the TaskManager that listens for additional Mesos-related * messages. @@ -32,7 +31,7 @@ import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerConfigurati class MesosTaskManager( config: TaskManagerConfiguration, resourceID: ResourceID, - connectionInfo: InstanceConnectionInfo, + taskManagerLocation: TaskManagerLocation, memoryManager: MemoryManager, ioManager: IOManager, network: NetworkEnvironment, @@ -41,7 +40,7 @@ class MesosTaskManager( extends TaskManager( config, resourceID, - connectionInfo, + taskManagerLocation, memoryManager, ioManager, network, diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandler.java index 7b6a36191f3fa..ce154e372cc3d 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandler.java @@ -21,7 +21,7 @@ import com.fasterxml.jackson.core.JsonGenerator; import org.apache.flink.runtime.executiongraph.ExecutionGraph; import org.apache.flink.runtime.executiongraph.ExecutionVertex; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; import org.apache.flink.util.ExceptionUtils; @@ -66,7 +66,7 @@ public String handleRequest(ExecutionGraph graph, Map params) th break; } - InstanceConnectionInfo location = task.getCurrentAssignedResourceLocation(); + TaskManagerLocation location = task.getCurrentAssignedResourceLocation(); String locationString = location != null ? location.getFQDNHostname() + ':' + location.dataPort() : "(unassigned)"; diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexDetailsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexDetailsHandler.java index d4e885e25aa81..813ecb82a260d 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexDetailsHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexDetailsHandler.java @@ -26,7 +26,7 @@ import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.executiongraph.ExecutionVertex; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; import java.io.StringWriter; @@ -61,7 +61,7 @@ public String handleRequest(ExecutionJobVertex jobVertex, Map pa for (ExecutionVertex vertex : jobVertex.getTaskVertices()) { final ExecutionState status = vertex.getExecutionState(); - InstanceConnectionInfo location = vertex.getCurrentAssignedResourceLocation(); + TaskManagerLocation location = vertex.getCurrentAssignedResourceLocation(); String locationString = location == null ? "(unassigned)" : location.getHostname() + ":" + location.dataPort(); long startTime = vertex.getStateTimestamp(ExecutionState.DEPLOYING); diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexTaskManagersHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexTaskManagersHandler.java index befc0bfa6509a..cbdb87f307444 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexTaskManagersHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexTaskManagersHandler.java @@ -25,7 +25,7 @@ import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.executiongraph.ExecutionVertex; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; import java.io.StringWriter; @@ -51,7 +51,7 @@ public String handleRequest(ExecutionJobVertex jobVertex, Map pa Map> taskManagerVertices = new HashMap<>(); for (ExecutionVertex vertex : jobVertex.getTaskVertices()) { - InstanceConnectionInfo location = vertex.getCurrentAssignedResourceLocation(); + TaskManagerLocation location = vertex.getCurrentAssignedResourceLocation(); String taskManager = location == null ? "(unassigned)" : location.getHostname() + ":" + location.dataPort(); List vertices = taskManagerVertices.get(taskManager); diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptDetailsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptDetailsHandler.java index 3d80b235c86f0..a1e6d0ed29679 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptDetailsHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptDetailsHandler.java @@ -25,7 +25,7 @@ import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.Execution; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; import java.io.StringWriter; @@ -45,7 +45,7 @@ public String handleRequest(Execution execAttempt, Map params) t final ExecutionState status = execAttempt.getState(); final long now = System.currentTimeMillis(); - InstanceConnectionInfo location = execAttempt.getAssignedResourceLocation(); + TaskManagerLocation location = execAttempt.getAssignedResourceLocation(); String locationString = location == null ? "(unassigned)" : location.getHostname(); long startTime = execAttempt.getStateTimestamp(ExecutionState.DEPLOYING); diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksAllAccumulatorsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksAllAccumulatorsHandler.java index 6d9ce3a49e15b..780bd4b696a99 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksAllAccumulatorsHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksAllAccumulatorsHandler.java @@ -23,7 +23,7 @@ import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.executiongraph.ExecutionVertex; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; import java.io.StringWriter; @@ -52,7 +52,7 @@ public String handleRequest(ExecutionJobVertex jobVertex, Map pa int num = 0; for (ExecutionVertex vertex : jobVertex.getTaskVertices()) { - InstanceConnectionInfo location = vertex.getCurrentAssignedResourceLocation(); + TaskManagerLocation location = vertex.getCurrentAssignedResourceLocation(); String locationString = location == null ? "(unassigned)" : location.getHostname(); gen.writeStartObject(); diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksTimesHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksTimesHandler.java index 03d40dc12e3dd..9e6276d1312e2 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksTimesHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksTimesHandler.java @@ -23,7 +23,7 @@ import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.executiongraph.ExecutionVertex; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; import java.io.StringWriter; @@ -70,7 +70,7 @@ public String handleRequest(ExecutionJobVertex jobVertex, Map pa gen.writeStartObject(); gen.writeNumberField("subtask", num++); - InstanceConnectionInfo location = vertex.getCurrentAssignedResourceLocation(); + TaskManagerLocation location = vertex.getCurrentAssignedResourceLocation(); String locationString = location == null ? "(unassigned)" : location.getHostname(); gen.writeStringField("host", locationString); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartialInputChannelDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartialInputChannelDeploymentDescriptor.java index a3cfcd934d718..e1391a46d7ea2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartialInputChannelDeploymentDescriptor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartialInputChannelDeploymentDescriptor.java @@ -21,7 +21,7 @@ import org.apache.flink.runtime.executiongraph.Execution; import org.apache.flink.runtime.executiongraph.IntermediateResult; import org.apache.flink.runtime.executiongraph.IntermediateResultPartition; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.io.network.ConnectionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; @@ -44,7 +44,7 @@ public class PartialInputChannelDeploymentDescriptor { private final ResultPartitionID partitionID; /** The partition connection info. */ - private final InstanceConnectionInfo partitionConnectionInfo; + private final TaskManagerLocation partitionConnectionInfo; /** The partition connection index. */ private final int partitionConnectionIndex; @@ -52,7 +52,7 @@ public class PartialInputChannelDeploymentDescriptor { public PartialInputChannelDeploymentDescriptor( IntermediateDataSetID resultId, ResultPartitionID partitionID, - InstanceConnectionInfo partitionConnectionInfo, + TaskManagerLocation partitionConnectionInfo, int partitionConnectionIndex) { this.resultId = checkNotNull(resultId); @@ -71,7 +71,7 @@ public InputChannelDeploymentDescriptor createInputChannelDeploymentDescriptor( checkNotNull(consumerExecution, "Consumer execution null"); - InstanceConnectionInfo consumerConnectionInfo = consumerExecution.getAssignedResourceLocation(); + TaskManagerLocation consumerConnectionInfo = consumerExecution.getAssignedResourceLocation(); checkNotNull(consumerConnectionInfo, "Consumer connection info null"); @@ -107,7 +107,7 @@ public static PartialInputChannelDeploymentDescriptor fromEdge( final IntermediateResult result = partition.getIntermediateResult(); final IntermediateDataSetID resultId = result.getId(); - final InstanceConnectionInfo partitionConnectionInfo = producer.getAssignedResourceLocation(); + final TaskManagerLocation partitionConnectionInfo = producer.getAssignedResourceLocation(); final int partitionConnectionIndex = result.getConnectionIndex(); return new PartialInputChannelDeploymentDescriptor( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java index efddecc143d22..197999ce37c92 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java @@ -31,7 +31,7 @@ import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.instance.Instance; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.io.network.ConnectionID; @@ -133,7 +133,7 @@ public class Execution { private volatile Throwable failureCause; // once assigned, never changes - private volatile InstanceConnectionInfo assignedResourceLocation; // for the archived execution + private volatile TaskManagerLocation assignedResourceLocation; // for the archived execution private ChainedStateHandle chainedStateHandle; @@ -147,7 +147,7 @@ public class Execution { /* Lock for updating the accumulators atomically. Prevents final accumulators to be overwritten * by partial accumulators on a late heartbeat*/ - private final SerializableObject accumulatorLock = new SerializableObject(); + private final Object accumulatorLock = new Object(); /* Continuously updated map of user-defined accumulators */ private volatile Map> userAccumulators; @@ -202,7 +202,7 @@ public SimpleSlot getAssignedResource() { return assignedResource; } - public InstanceConnectionInfo getAssignedResourceLocation() { + public TaskManagerLocation getAssignedResourceLocation() { return assignedResourceLocation; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java index b2153940f80f6..e5a115a6eed69 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java @@ -28,7 +28,7 @@ import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.instance.Instance; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; @@ -228,7 +228,7 @@ public SimpleSlot getCurrentAssignedResource() { return currentExecution.getAssignedResource(); } - public InstanceConnectionInfo getCurrentAssignedResourceLocation() { + public TaskManagerLocation getCurrentAssignedResourceLocation() { return currentExecution.getAssignedResourceLocation(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java index 119f060315835..598b32bbc89ac 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java @@ -28,6 +28,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.jobmanager.scheduler.SlotAvailabilityListener; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,7 +47,7 @@ public class Instance { private final ActorGateway actorGateway; /** The instance connection information for the data transfer. */ - private final InstanceConnectionInfo connectionInfo; + private final TaskManagerLocation connectionInfo; /** A description of the resources of the task manager */ private final HardwareDescription resources; @@ -92,7 +93,7 @@ public class Instance { */ public Instance( ActorGateway actorGateway, - InstanceConnectionInfo connectionInfo, + TaskManagerLocation connectionInfo, ResourceID resourceId, InstanceID id, HardwareDescription resources, @@ -350,7 +351,7 @@ public ActorGateway getActorGateway() { return actorGateway; } - public InstanceConnectionInfo getInstanceConnectionInfo() { + public TaskManagerLocation getInstanceConnectionInfo() { return connectionInfo; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java index 0d0d4c7ff6923..e7a45376d4631 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java @@ -29,6 +29,7 @@ import akka.actor.ActorRef; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -147,7 +148,7 @@ public boolean reportHeartBeat(InstanceID instanceId, byte[] lastMetricsReport) public InstanceID registerTaskManager( ActorRef taskManager, ResourceID resourceID, - InstanceConnectionInfo connectionInfo, + TaskManagerLocation connectionInfo, HardwareDescription resources, int numberOfSlots, UUID leaderSessionID){ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionID.java index 0569daee9d90d..cc2a19db00a41 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionID.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionID.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.io.network; import org.apache.flink.runtime.executiongraph.IntermediateResult; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import java.io.Serializable; import java.net.InetSocketAddress; @@ -43,7 +43,7 @@ public class ConnectionID implements Serializable { private final int connectionIndex; - public ConnectionID(InstanceConnectionInfo connectionInfo, int connectionIndex) { + public ConnectionID(TaskManagerLocation connectionInfo, int connectionIndex) { this(new InetSocketAddress(connectionInfo.address(), connectionInfo.dataPort()), connectionIndex); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceConnectionInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerLocation.java similarity index 61% rename from flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceConnectionInfo.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerLocation.java index 2830f04043b28..5a0faa55a72d1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceConnectionInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerLocation.java @@ -16,54 +16,53 @@ * limitations under the License. */ -package org.apache.flink.runtime.instance; +package org.apache.flink.runtime.taskmanager; -import java.io.IOException; -import java.io.Serializable; import java.net.InetAddress; -import java.net.UnknownHostException; -import org.apache.flink.core.io.IOReadableWritable; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.util.NetUtils; -import org.apache.flink.util.StringUtils; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nonnull; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * This class encapsulates the connection information of a TaskManager. * It describes the host where the TaskManager operates and its server port * for data exchange. This class also contains utilities to work with the * TaskManager's host name, which is used to localize work assignments. */ -public class InstanceConnectionInfo implements IOReadableWritable, Comparable, Serializable { +public class TaskManagerLocation implements Comparable, java.io.Serializable { private static final long serialVersionUID = -8254407801276350716L; - - private static final Logger LOG = LoggerFactory.getLogger(InstanceConnectionInfo.class); - - /** - * The network address the instance's task manager binds its sockets to. - */ - private InetAddress inetAddress; + private static final Logger LOG = LoggerFactory.getLogger(TaskManagerLocation.class); - /** - * The port the instance's task manager expects to receive transfer envelopes on. - */ - private int dataPort; + // ------------------------------------------------------------------------ - /** - * The fully qualified host name of the instance. - */ - private String fqdnHostName; + /** The ID of the resource in which the TaskManager is started. This can be for example + * the YARN container ID, Mesos container ID, or any other unique identifier. */ + private final ResourceID resourceID; + + /** The network address that the TaskManager binds its sockets to */ + private final InetAddress inetAddress; + + /** The fully qualified host name of the TaskManager */ + private final String fqdnHostName; + + /** The pure hostname, derived from the fully qualified host name. */ + private final String hostName; - /** - * The hostname, derived from the fully qualified host name. - */ - private String hostName; + /** The port that the TaskManager receive data transport connection requests at */ + private final int dataPort; + /** The toString representation, eagerly constructed and cached to avoid repeated string building */ + private final String stringRepresentation; /** * Constructs a new instance connection info object. The constructor will attempt to retrieve the instance's @@ -74,29 +73,26 @@ public class InstanceConnectionInfo implements IOReadableWritable, Comparable 0 || dataPort == -1, "dataPort must be > 0, or -1 (local)"); + this.resourceID = checkNotNull(resourceID); + this.inetAddress = checkNotNull(inetAddress); this.dataPort = dataPort; - this.inetAddress = inetAddress; - + // get FQDN hostname on this TaskManager. + String fqdnHostName; try { - this.fqdnHostName = this.inetAddress.getCanonicalHostName(); + fqdnHostName = this.inetAddress.getCanonicalHostName(); } catch (Throwable t) { LOG.warn("Unable to determine the canonical hostname. Input split assignment (such as " + "for HDFS files) may be non-local when the canonical hostname is missing."); LOG.debug("getCanonicalHostName() Exception:", t); - this.fqdnHostName = this.inetAddress.getHostAddress(); + fqdnHostName = this.inetAddress.getHostAddress(); } + this.fqdnHostName = fqdnHostName; if (this.fqdnHostName.equals(this.inetAddress.getHostAddress())) { // this happens when the name lookup fails, either due to an exception, @@ -110,13 +106,30 @@ public InstanceConnectionInfo(InetAddress inetAddress, int dataPort) { else { this.hostName = NetUtils.getHostnameFromFQDN(this.fqdnHostName); } + + this.stringRepresentation = String.format( + "TaskManager (%s) @ %s (dataPort=%d)", resourceID, fqdnHostName, dataPort); } + // ------------------------------------------------------------------------ + // Getters + // ------------------------------------------------------------------------ + /** - * Constructs an empty object. + * Gets the ID of the resource in which the TaskManager is started. The format of this depends + * on how the TaskManager is started: + *

    + *
  • If the TaskManager is started via YARN, this is the YARN container ID.
  • + *
  • If the TaskManager is started via Mesos, this is the Mesos container ID.
  • + *
  • If the TaskManager is started in standalone mode, or via a MiniCluster, this is a random ID.
  • + *
  • Other deployment modes can set the resource ID in other ways.
  • + *
+ * + * @return The ID of the resource in which the TaskManager is started */ - public InstanceConnectionInfo() {} - + public ResourceID getResourceID() { + return resourceID; + } /** * Returns the port instance's task manager expects to receive transfer envelopes on. @@ -124,7 +137,7 @@ public InstanceConnectionInfo() {} * @return the port instance's task manager expects to receive transfer envelopes on */ public int dataPort() { - return this.dataPort; + return dataPort; } /** @@ -133,7 +146,16 @@ public int dataPort() { * @return the network address the instance's task manager binds its sockets to */ public InetAddress address() { - return this.inetAddress; + return inetAddress; + } + + /** + * Gets the IP address where the TaskManager operates. + * + * @return The IP address. + */ + public String addressString() { + return inetAddress.toString(); } /** @@ -143,7 +165,7 @@ public InetAddress address() { * @return The fully-qualified domain name of the TaskManager. */ public String getFQDNHostname() { - return this.fqdnHostName; + return fqdnHostName; } /** @@ -163,82 +185,50 @@ public String getHostname() { return hostName; } - /** - * Gets the IP address where the TaskManager operates. - * - * @return The IP address. - */ - public String getInetAdress() { - return this.inetAddress.toString(); - } - - // -------------------------------------------------------------------------------------------- - // Serialization - // -------------------------------------------------------------------------------------------- - - @Override - public void read(DataInputView in) throws IOException { - - final int addr_length = in.readInt(); - byte[] address = new byte[addr_length]; - in.readFully(address); - - this.dataPort = in.readInt(); - - this.fqdnHostName = StringUtils.readNullableString(in); - this.hostName = StringUtils.readNullableString(in); - - try { - this.inetAddress = InetAddress.getByAddress(address); - } catch (UnknownHostException e) { - throw new IOException("This lookup should never fail.", e); - } - } - - - @Override - public void write(final DataOutputView out) throws IOException { - out.writeInt(this.inetAddress.getAddress().length); - out.write(this.inetAddress.getAddress()); - - out.writeInt(this.dataPort); - - StringUtils.writeNullableString(fqdnHostName, out); - StringUtils.writeNullableString(hostName, out); - } - // -------------------------------------------------------------------------------------------- // Utilities // -------------------------------------------------------------------------------------------- @Override public String toString() { - return getFQDNHostname() + " (dataPort=" + dataPort + ")"; + return stringRepresentation; } @Override public boolean equals(Object obj) { - if (obj instanceof InstanceConnectionInfo) { - InstanceConnectionInfo other = (InstanceConnectionInfo) obj; - return this.dataPort == other.dataPort && - this.inetAddress.equals(other.inetAddress); - } else { + if (obj == this) { + return true; + } + else if (obj != null && obj.getClass() == TaskManagerLocation.class) { + TaskManagerLocation that = (TaskManagerLocation) obj; + return this.resourceID.equals(that.resourceID) && + this.inetAddress.equals(that.inetAddress) && + this.dataPort == that.dataPort; + } + else { return false; } } @Override public int hashCode() { - return this.inetAddress.hashCode() + - 17*dataPort; + return resourceID.hashCode() + + 17 * inetAddress.hashCode() + + 129 * dataPort; } @Override - public int compareTo(InstanceConnectionInfo o) { + public int compareTo(@Nonnull TaskManagerLocation o) { // decide based on address first + int resourceIdCmp = this.resourceID.getResourceIdString().compareTo(o.resourceID.getResourceIdString()); + if (resourceIdCmp != 0) { + return resourceIdCmp; + } + + // decide based on ip address next byte[] thisAddress = this.inetAddress.getAddress(); byte[] otherAddress = o.inetAddress.getAddress(); - + if (thisAddress.length < otherAddress.length) { return -1; } else if (thisAddress.length > otherAddress.length) { @@ -254,7 +244,7 @@ public int compareTo(InstanceConnectionInfo o) { } } } - + // addresses are identical, decide based on ports. if (this.dataPort < o.dataPort) { return -1; diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/RegistrationMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/RegistrationMessages.scala index d362164eb4ce6..5648bc69f9e5f 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/RegistrationMessages.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/RegistrationMessages.scala @@ -20,9 +20,9 @@ package org.apache.flink.runtime.messages import java.util.UUID -import akka.actor.ActorRef import org.apache.flink.runtime.clusterframework.types.ResourceID -import org.apache.flink.runtime.instance.{HardwareDescription, InstanceConnectionInfo, InstanceID} +import org.apache.flink.runtime.instance.{HardwareDescription, InstanceID} +import org.apache.flink.runtime.taskmanager.TaskManagerLocation import scala.concurrent.duration.{Deadline, FiniteDuration} @@ -63,10 +63,10 @@ object RegistrationMessages { * @param numberOfSlots The number of processing slots offered by the TaskManager. */ case class RegisterTaskManager( - resourceId: ResourceID, - connectionInfo: InstanceConnectionInfo, - resources: HardwareDescription, - numberOfSlots: Int) + resourceId: ResourceID, + connectionInfo: TaskManagerLocation, + resources: HardwareDescription, + numberOfSlots: Int) extends RegistrationMessage /** diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index 31548263ff871..84750a389af8a 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -50,7 +50,7 @@ import org.apache.flink.runtime.execution.ExecutionState import org.apache.flink.runtime.execution.librarycache.{BlobLibraryCacheManager, FallbackLibraryCacheManager, LibraryCacheManager} import org.apache.flink.runtime.executiongraph.ExecutionAttemptID import org.apache.flink.runtime.filecache.FileCache -import org.apache.flink.runtime.instance.{AkkaActorGateway, HardwareDescription, InstanceConnectionInfo, InstanceID} +import org.apache.flink.runtime.instance.{AkkaActorGateway, HardwareDescription, InstanceID} import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode import org.apache.flink.runtime.io.disk.iomanager.{IOManager, IOManagerAsync} import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool @@ -127,7 +127,7 @@ import scala.util.{Failure, Success} class TaskManager( protected val config: TaskManagerConfiguration, protected val resourceID: ResourceID, - protected val connectionInfo: InstanceConnectionInfo, + protected val location: TaskManagerLocation, protected val memoryManager: MemoryManager, protected val ioManager: IOManager, protected val network: NetworkEnvironment, @@ -189,7 +189,7 @@ class TaskManager( var leaderSessionID: Option[UUID] = None private val runtimeInfo = new TaskManagerRuntimeInfo( - connectionInfo.getHostname(), + location.getHostname(), new UnmodifiableConfiguration(config.configuration), config.tmpDirPaths) @@ -209,7 +209,7 @@ class TaskManager( */ override def preStart(): Unit = { log.info(s"Starting TaskManager actor at ${self.path.toSerializationFormat}.") - log.info(s"TaskManager data connection information: $connectionInfo") + log.info(s"TaskManager data connection information: $location") log.info(s"TaskManager has $numberOfSlots task slot(s).") // log the initial memory utilization @@ -601,7 +601,7 @@ class TaskManager( jobManager ! decorateMessage( RegisterTaskManager( resourceID, - connectionInfo, + location, resources, numberOfSlots) ) @@ -1884,7 +1884,8 @@ object TaskManager { network.start() - val connectionInfo = new InstanceConnectionInfo( + val taskManagerLocation = new TaskManagerLocation( + resourceID, taskManagerAddress.getAddress(), network.getConnectionManager().getDataPort()) @@ -1994,7 +1995,7 @@ object TaskManager { taskManagerClass, taskManagerConfig, resourceID, - connectionInfo, + taskManagerLocation, memoryManager, ioManager, network, diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala index 2597753e15f90..9b5a147360389 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala @@ -19,26 +19,25 @@ package org.apache.flink.runtime.testingUtils import org.apache.flink.runtime.clusterframework.types.ResourceID -import org.apache.flink.runtime.instance.InstanceConnectionInfo import org.apache.flink.runtime.io.disk.iomanager.IOManager import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService import org.apache.flink.runtime.memory.MemoryManager -import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerConfiguration} +import org.apache.flink.runtime.taskmanager.{TaskManagerLocation, TaskManager, TaskManagerConfiguration} import scala.language.postfixOps /** Subclass of the [[TaskManager]] to support testing messages */ class TestingTaskManager( - config: TaskManagerConfiguration, - resourceID: ResourceID, - connectionInfo: InstanceConnectionInfo, - memoryManager: MemoryManager, - ioManager: IOManager, - network: NetworkEnvironment, - numberOfSlots: Int, - leaderRetrievalService: LeaderRetrievalService) + config: TaskManagerConfiguration, + resourceID: ResourceID, + connectionInfo: TaskManagerLocation, + memoryManager: MemoryManager, + ioManager: IOManager, + network: NetworkEnvironment, + numberOfSlots: Int, + leaderRetrievalService: LeaderRetrievalService) extends TaskManager( config, resourceID, @@ -51,13 +50,13 @@ class TestingTaskManager( with TestingTaskManagerLike { def this( - config: TaskManagerConfiguration, - connectionInfo: InstanceConnectionInfo, - memoryManager: MemoryManager, - ioManager: IOManager, - network: NetworkEnvironment, - numberOfSlots: Int, - leaderRetrievalService: LeaderRetrievalService) { + config: TaskManagerConfiguration, + connectionInfo: TaskManagerLocation, + memoryManager: MemoryManager, + ioManager: IOManager, + network: NetworkEnvironment, + numberOfSlots: Int, + leaderRetrievalService: LeaderRetrievalService) { this( config, ResourceID.generate(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java index 2b8b867d5ee0f..cf7cf5800ec7c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java @@ -32,7 +32,7 @@ import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.instance.Instance; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.instance.Slot; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -110,7 +110,7 @@ public void testExecutionGraphRestartTimeMetric() throws JobException, IOExcepti Instance instance = mock(Instance.class); - InstanceConnectionInfo instanceConnectionInfo = mock(InstanceConnectionInfo.class); + TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class); Slot rootSlot = mock(Slot.class); @@ -123,9 +123,9 @@ public void testExecutionGraphRestartTimeMetric() throws JobException, IOExcepti when(scheduler.scheduleImmediately(Matchers.any(ScheduledUnit.class))).thenReturn(simpleSlot); - when(instance.getInstanceConnectionInfo()).thenReturn(instanceConnectionInfo); + when(instance.getInstanceConnectionInfo()).thenReturn(taskManagerLocation); when(instance.getActorGateway()).thenReturn(actorGateway); - when(instanceConnectionInfo.getHostname()).thenReturn("localhost"); + when(taskManagerLocation.getHostname()).thenReturn("localhost"); when(rootSlot.getSlotNumber()).thenReturn(0); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java index 903d5f943be2c..cddb6cbf61502 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java @@ -35,7 +35,7 @@ import org.apache.flink.runtime.instance.BaseTestingActorGateway; import org.apache.flink.runtime.instance.HardwareDescription; import org.apache.flink.runtime.instance.Instance; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.instance.SimpleSlot; @@ -108,11 +108,12 @@ public static Instance getInstance(final ActorGateway gateway) throws Exception } public static Instance getInstance(final ActorGateway gateway, final int numberOfSlots) throws Exception { + ResourceID resourceID = ResourceID.generate(); HardwareDescription hardwareDescription = new HardwareDescription(4, 2L*1024*1024*1024, 1024*1024*1024, 512*1024*1024); InetAddress address = InetAddress.getByName("127.0.0.1"); - InstanceConnectionInfo connection = new InstanceConnectionInfo(address, 10001); + TaskManagerLocation connection = new TaskManagerLocation(resourceID, address, 10001); - return new Instance(gateway, connection, ResourceID.generate(), new InstanceID(), hardwareDescription, numberOfSlots); + return new Instance(gateway, connection, resourceID, new InstanceID(), hardwareDescription, numberOfSlots); } @SuppressWarnings("serial") diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java index 2a690d99359a4..a71faf62eedc8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java @@ -27,7 +27,7 @@ import org.apache.flink.runtime.instance.DummyActorGateway; import org.apache.flink.runtime.instance.HardwareDescription; import org.apache.flink.runtime.instance.Instance; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.jobgraph.JobVertex; @@ -79,12 +79,13 @@ public TerminalStateDeadlockTest() { this.execGraphSchedulerField.setAccessible(true); // the dummy resource + ResourceID resourceId = ResourceID.generate(); InetAddress address = InetAddress.getByName("127.0.0.1"); - InstanceConnectionInfo ci = new InstanceConnectionInfo(address, 12345); + TaskManagerLocation ci = new TaskManagerLocation(resourceId, address, 12345); HardwareDescription resources = new HardwareDescription(4, 4000000, 3000000, 2000000); Instance instance = new Instance(DummyActorGateway.INSTANCE, ci, - ResourceID.generate(), new InstanceID(), resources, 4); + resourceId, new InstanceID(), resources, 4); this.resource = instance.allocateSimpleSlot(new JobID()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java index 4ee06b364c4ba..91472ae2fdbb5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java @@ -32,7 +32,7 @@ import org.apache.flink.runtime.instance.DummyActorGateway; import org.apache.flink.runtime.instance.HardwareDescription; import org.apache.flink.runtime.instance.Instance; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.jobgraph.JobVertex; @@ -443,10 +443,10 @@ public void testArchivingClearsFields() { public static Instance getInstance(byte[] ipAddress, int dataPort, String hostname) throws Exception { HardwareDescription hardwareDescription = new HardwareDescription(4, 2L*1024*1024*1024, 1024*1024*1024, 512*1024*1024); - InstanceConnectionInfo connection = mock(InstanceConnectionInfo.class); + TaskManagerLocation connection = mock(TaskManagerLocation.class); when(connection.address()).thenReturn(InetAddress.getByAddress(ipAddress)); when(connection.dataPort()).thenReturn(dataPort); - when(connection.getInetAdress()).thenReturn(InetAddress.getByAddress(ipAddress).toString()); + when(connection.addressString()).thenReturn(InetAddress.getByAddress(ipAddress).toString()); when(connection.getHostname()).thenReturn(hostname); when(connection.getFQDNHostname()).thenReturn(hostname); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java index ff5e2ab4ede96..f1ed9601e8ec3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java @@ -18,24 +18,11 @@ package org.apache.flink.runtime.instance; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - - -import java.net.InetAddress; -import java.util.Collection; -import java.util.HashSet; -import java.util.Iterator; -import java.util.Set; -import java.util.UUID; - import akka.actor.ActorSystem; import akka.testkit.JavaTestKit; + import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testutils.CommonTestUtils; @@ -44,6 +31,18 @@ import org.junit.BeforeClass; import org.junit.Test; +import java.net.InetAddress; +import java.util.Collection; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Set; +import java.util.UUID; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + /** * Tests for {@link org.apache.flink.runtime.instance.InstanceManager}. */ @@ -76,13 +75,13 @@ public void testInstanceRegistering() { InetAddress address = InetAddress.getByName("127.0.0.1"); // register three instances - InstanceConnectionInfo ici1 = new InstanceConnectionInfo(address, dataPort); - InstanceConnectionInfo ici2 = new InstanceConnectionInfo(address, dataPort + 15); - InstanceConnectionInfo ici3 = new InstanceConnectionInfo(address, dataPort + 30); - ResourceID resID1 = ResourceID.generate(); ResourceID resID2 = ResourceID.generate(); ResourceID resID3 = ResourceID.generate(); + + TaskManagerLocation ici1 = new TaskManagerLocation(resID1, address, dataPort); + TaskManagerLocation ici2 = new TaskManagerLocation(resID2, address, dataPort + 15); + TaskManagerLocation ici3 = new TaskManagerLocation(resID3, address, dataPort + 30); final JavaTestKit probe1 = new JavaTestKit(system); final JavaTestKit probe2 = new JavaTestKit(system); @@ -99,16 +98,16 @@ public void testInstanceRegistering() { assertEquals(8, cm.getTotalNumberOfSlots()); Collection instances = cm.getAllRegisteredInstances(); - Set instanceConnectionInfos = new - HashSet(); + Set taskManagerLocations = new + HashSet(); for(Instance instance: instances){ - instanceConnectionInfos.add(instance.getInstanceConnectionInfo()); + taskManagerLocations.add(instance.getInstanceConnectionInfo()); } - assertTrue(instanceConnectionInfos.contains(ici1)); - assertTrue(instanceConnectionInfos.contains(ici2)); - assertTrue(instanceConnectionInfos.contains(ici3)); + assertTrue(taskManagerLocations.contains(ici1)); + assertTrue(taskManagerLocations.contains(ici2)); + assertTrue(taskManagerLocations.contains(ici3)); cm.shutdown(); } @@ -131,7 +130,7 @@ public void testRegisteringAlreadyRegistered() { HardwareDescription resources = HardwareDescription.extractFromSystem(4096); InetAddress address = InetAddress.getByName("127.0.0.1"); - InstanceConnectionInfo ici = new InstanceConnectionInfo(address, dataPort); + TaskManagerLocation ici = new TaskManagerLocation(resID1, address, dataPort); JavaTestKit probe = new JavaTestKit(system); cm.registerTaskManager(probe.getRef(), resID1, @@ -141,13 +140,12 @@ public void testRegisteringAlreadyRegistered() { assertEquals(1, cm.getTotalNumberOfSlots()); try { - cm.registerTaskManager(probe.getRef(), resID2, - ici, resources, 1, leaderSessionID); + cm.registerTaskManager(probe.getRef(), resID2, ici, resources, 1, leaderSessionID); } catch (Exception e) { // good } - // check for correct number of registerede instances + // check for correct number of registered instances assertEquals(1, cm.getNumberOfRegisteredTaskManagers()); assertEquals(1, cm.getTotalNumberOfSlots()); @@ -176,9 +174,9 @@ public void testReportHeartbeat() { InetAddress address = InetAddress.getByName("127.0.0.1"); // register three instances - InstanceConnectionInfo ici1 = new InstanceConnectionInfo(address, dataPort); - InstanceConnectionInfo ici2 = new InstanceConnectionInfo(address, dataPort + 1); - InstanceConnectionInfo ici3 = new InstanceConnectionInfo(address, dataPort + 2); + TaskManagerLocation ici1 = new TaskManagerLocation(resID1, address, dataPort); + TaskManagerLocation ici2 = new TaskManagerLocation(resID2, address, dataPort + 1); + TaskManagerLocation ici3 = new TaskManagerLocation(resID3, address, dataPort + 2); JavaTestKit probe1 = new JavaTestKit(system); JavaTestKit probe2 = new JavaTestKit(system); @@ -240,7 +238,7 @@ public void testShutdown() { ResourceID resID = ResourceID.generate(); HardwareDescription resources = HardwareDescription.extractFromSystem(4096); InetAddress address = InetAddress.getByName("127.0.0.1"); - InstanceConnectionInfo ici = new InstanceConnectionInfo(address, 20000); + TaskManagerLocation ici = new TaskManagerLocation(resID, address, 20000); JavaTestKit probe = new JavaTestKit(system); cm.registerTaskManager(probe.getRef(), resID, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceTest.java index faa679b6337f8..82d3723b92d98 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceTest.java @@ -25,6 +25,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.junit.Test; /** @@ -35,12 +36,13 @@ public class InstanceTest { @Test public void testAllocatingAndCancellingSlots() { try { + ResourceID resourceID = ResourceID.generate(); HardwareDescription hardwareDescription = new HardwareDescription(4, 2L*1024*1024*1024, 1024*1024*1024, 512*1024*1024); InetAddress address = InetAddress.getByName("127.0.0.1"); - InstanceConnectionInfo connection = new InstanceConnectionInfo(address, 10001); + TaskManagerLocation connection = new TaskManagerLocation(resourceID, address, 10001); Instance instance = new Instance(DummyActorGateway.INSTANCE, connection, - ResourceID.generate(), new InstanceID(), hardwareDescription, 4); + resourceID, new InstanceID(), hardwareDescription, 4); assertEquals(4, instance.getTotalNumberOfSlots()); assertEquals(4, instance.getNumberOfAvailableSlots()); @@ -97,12 +99,13 @@ public void testAllocatingAndCancellingSlots() { @Test public void testInstanceDies() { try { + ResourceID resourceID = ResourceID.generate(); HardwareDescription hardwareDescription = new HardwareDescription(4, 2L*1024*1024*1024, 1024*1024*1024, 512*1024*1024); InetAddress address = InetAddress.getByName("127.0.0.1"); - InstanceConnectionInfo connection = new InstanceConnectionInfo(address, 10001); + TaskManagerLocation connection = new TaskManagerLocation(resourceID, address, 10001); Instance instance = new Instance(DummyActorGateway.INSTANCE, connection, - ResourceID.generate(), new InstanceID(), hardwareDescription, 3); + resourceID, new InstanceID(), hardwareDescription, 3); assertEquals(3, instance.getNumberOfAvailableSlots()); @@ -128,12 +131,13 @@ public void testInstanceDies() { @Test public void testCancelAllSlots() { try { + ResourceID resourceID = ResourceID.generate(); HardwareDescription hardwareDescription = new HardwareDescription(4, 2L*1024*1024*1024, 1024*1024*1024, 512*1024*1024); InetAddress address = InetAddress.getByName("127.0.0.1"); - InstanceConnectionInfo connection = new InstanceConnectionInfo(address, 10001); + TaskManagerLocation connection = new TaskManagerLocation(resourceID, address, 10001); Instance instance = new Instance(DummyActorGateway.INSTANCE, connection, - ResourceID.generate(), new InstanceID(), hardwareDescription, 3); + resourceID, new InstanceID(), hardwareDescription, 3); assertEquals(3, instance.getNumberOfAvailableSlots()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotTest.java index 459a3ed613096..82c2a740cd620 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotTest.java @@ -27,6 +27,7 @@ import org.apache.flink.runtime.executiongraph.Execution; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.junit.Test; import org.mockito.Matchers; @@ -143,12 +144,13 @@ public void testReleaseCancelsVertex() { } public static SimpleSlot getSlot() throws Exception { + ResourceID resourceID = ResourceID.generate(); HardwareDescription hardwareDescription = new HardwareDescription(4, 2L*1024*1024*1024, 1024*1024*1024, 512*1024*1024); InetAddress address = InetAddress.getByName("127.0.0.1"); - InstanceConnectionInfo connection = new InstanceConnectionInfo(address, 10001); + TaskManagerLocation connection = new TaskManagerLocation(resourceID, address, 10001); Instance instance = new Instance(DummyActorGateway.INSTANCE, connection, - ResourceID.generate(), new InstanceID(), hardwareDescription, 1); + resourceID, new InstanceID(), hardwareDescription, 1); return instance.allocateSimpleSlot(new JobID()); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java index 983d6e668c858..99360e24f012f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java @@ -24,7 +24,6 @@ import java.net.InetAddress; import java.net.UnknownHostException; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.concurrent.atomic.AtomicInteger; @@ -35,7 +34,7 @@ import org.apache.flink.runtime.instance.DummyActorGateway; import org.apache.flink.runtime.instance.HardwareDescription; import org.apache.flink.runtime.instance.Instance; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.jobgraph.JobVertexID; @@ -52,7 +51,8 @@ public static Instance getRandomInstance(int numSlots) { throw new IllegalArgumentException(); } - InetAddress address; + final ResourceID resourceID = ResourceID.generate(); + final InetAddress address; try { address = InetAddress.getByName("127.0.0.1"); } @@ -62,12 +62,12 @@ public static Instance getRandomInstance(int numSlots) { int dataPort = port.getAndIncrement(); - InstanceConnectionInfo ci = new InstanceConnectionInfo(address, dataPort); + TaskManagerLocation ci = new TaskManagerLocation(resourceID, address, dataPort); final long GB = 1024L*1024*1024; HardwareDescription resources = new HardwareDescription(4, 4*GB, 3*GB, 2*GB); - return new Instance(DummyActorGateway.INSTANCE, ci, ResourceID.generate(), + return new Instance(DummyActorGateway.INSTANCE, ci, resourceID, new InstanceID(), resources, numSlots); } @@ -143,19 +143,4 @@ public static boolean areAllDistinct(Object ... obj) { return set.size() == obj.length; } - - public static boolean areSameSets(Collection set1, Collection set2) { - if (set1 == null || set2 == null) { - throw new IllegalArgumentException(); - } - - HashSet set = new HashSet(set1); - for (Object o : set2) { - if (!set.remove(o)) { - return false; - } - } - - return set.isEmpty(); - } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerITCase.java index ca09634a79124..3307568bf99ef 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerITCase.java @@ -24,14 +24,13 @@ import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.instance.ActorGateway; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.messages.Messages; import org.apache.flink.runtime.messages.RegistrationMessages; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testutils.TestingResourceManager; import org.apache.flink.util.TestLogger; import org.junit.AfterClass; -import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; import org.mockito.Mockito; @@ -83,7 +82,7 @@ protected void run() { jobManager.tell( new RegistrationMessages.RegisterTaskManager( resourceID, - Mockito.mock(InstanceConnectionInfo.class), + Mockito.mock(TaskManagerLocation.class), null, 1), me); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java index 3371c49bde702..bda41748432ea 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java @@ -25,6 +25,7 @@ import akka.actor.Kill; import akka.actor.Props; import akka.testkit.JavaTestKit; + import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.memory.MemoryType; @@ -32,7 +33,6 @@ import org.apache.flink.runtime.clusterframework.FlinkResourceManager; import org.apache.flink.runtime.clusterframework.standalone.StandaloneResourceManager; import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.io.network.LocalConnectionManager; @@ -46,12 +46,12 @@ import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService; import org.apache.flink.runtime.memory.MemoryManager; - import org.apache.flink.runtime.messages.TaskManagerMessages; import org.apache.flink.runtime.query.KvStateRegistry; -import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.util.LeaderRetrievalUtils; + import org.junit.Test; + import scala.Option; import scala.concurrent.duration.FiniteDuration; @@ -105,7 +105,9 @@ public void testComponentsStartupShutdown() { 32, BUFFER_SIZE, MemoryType.HEAP, IOManager.IOMode.SYNC, 0, 0, 0, Option.empty(), 0, 0); - final InstanceConnectionInfo connectionInfo = new InstanceConnectionInfo(InetAddress.getLocalHost(), 10000); + ResourceID taskManagerId = ResourceID.generate(); + + final TaskManagerLocation connectionInfo = new TaskManagerLocation(taskManagerId, InetAddress.getLocalHost(), 10000); final MemoryManager memManager = new MemoryManager(32 * BUFFER_SIZE, 1, BUFFER_SIZE, MemoryType.HEAP, false); final IOManager ioManager = new IOManagerAsync(TMP_DIR); @@ -130,7 +132,7 @@ public void testComponentsStartupShutdown() { final Props tmProps = Props.create( TaskManager.class, tmConfig, - ResourceID.generate(), + taskManagerId, connectionInfo, memManager, ioManager, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceConnectionInfoTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerLocationTest.java similarity index 75% rename from flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceConnectionInfoTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerLocationTest.java index 3a9488d2a0a1b..9452b20fee967 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceConnectionInfoTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerLocationTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.instance; +package org.apache.flink.runtime.taskmanager; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertEquals; @@ -25,6 +25,7 @@ import java.net.InetAddress; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.util.InstantiationUtil; import org.junit.Assert; @@ -34,14 +35,18 @@ import static org.mockito.Mockito.when; /** - * Tests for the InstanceConnectionInfo, which identifies the location and connection + * Tests for the TaskManagerLocation, which identifies the location and connection * information of a TaskManager. */ -public class InstanceConnectionInfoTest { +public class TaskManagerLocationTest { @Test public void testEqualsHashAndCompareTo() { try { + ResourceID resourceID1 = new ResourceID("a"); + ResourceID resourceID2 = new ResourceID("b"); + ResourceID resourceID3 = new ResourceID("c"); + // we mock the addresses to save the times of the reverse name lookups InetAddress address1 = mock(InetAddress.class); when(address1.getCanonicalHostName()).thenReturn("localhost"); @@ -62,10 +67,10 @@ public void testEqualsHashAndCompareTo() { when(address3.getAddress()).thenReturn(new byte[] {(byte) 192, (byte) 168, 0, 1} ); // one == four != two != three - InstanceConnectionInfo one = new InstanceConnectionInfo(address1, 19871); - InstanceConnectionInfo two = new InstanceConnectionInfo(address2, 19871); - InstanceConnectionInfo three = new InstanceConnectionInfo(address3, 10871); - InstanceConnectionInfo four = new InstanceConnectionInfo(address1, 19871); + TaskManagerLocation one = new TaskManagerLocation(resourceID1, address1, 19871); + TaskManagerLocation two = new TaskManagerLocation(resourceID2, address2, 19871); + TaskManagerLocation three = new TaskManagerLocation(resourceID3, address3, 10871); + TaskManagerLocation four = new TaskManagerLocation(resourceID1, address1, 19871); assertTrue(one.equals(four)); assertTrue(!one.equals(two)); @@ -96,24 +101,20 @@ public void testSerialization() { try { // without resolved hostname { - InstanceConnectionInfo original = new InstanceConnectionInfo(InetAddress.getByName("1.2.3.4"), 8888); - - InstanceConnectionInfo copy = InstantiationUtil.createCopyWritable(original); - assertEquals(original, copy); - - InstanceConnectionInfo serCopy = InstantiationUtil.clone(original); + TaskManagerLocation original = new TaskManagerLocation( + ResourceID.generate(), InetAddress.getByName("1.2.3.4"), 8888); + + TaskManagerLocation serCopy = InstantiationUtil.clone(original); assertEquals(original, serCopy); } // with resolved hostname { - InstanceConnectionInfo original = new InstanceConnectionInfo(InetAddress.getByName("127.0.0.1"), 19871); + TaskManagerLocation original = new TaskManagerLocation( + ResourceID.generate(), InetAddress.getByName("127.0.0.1"), 19871); original.getFQDNHostname(); - - InstanceConnectionInfo copy = InstantiationUtil.createCopyWritable(original); - assertEquals(original, copy); - - InstanceConnectionInfo serCopy = InstantiationUtil.clone(original); + + TaskManagerLocation serCopy = InstantiationUtil.clone(original); assertEquals(original, serCopy); } } @@ -126,10 +127,10 @@ public void testSerialization() { @Test public void testGetFQDNHostname() { try { - InstanceConnectionInfo info1 = new InstanceConnectionInfo(InetAddress.getByName("127.0.0.1"), 19871); + TaskManagerLocation info1 = new TaskManagerLocation(ResourceID.generate(), InetAddress.getByName("127.0.0.1"), 19871); assertNotNull(info1.getFQDNHostname()); - InstanceConnectionInfo info2 = new InstanceConnectionInfo(InetAddress.getByName("1.2.3.4"), 8888); + TaskManagerLocation info2 = new TaskManagerLocation(ResourceID.generate(), InetAddress.getByName("1.2.3.4"), 8888); assertNotNull(info2.getFQDNHostname()); } catch (Exception e) { @@ -146,7 +147,7 @@ public void testGetHostname0() { when(address.getHostName()).thenReturn("worker2.cluster.mycompany.com"); when(address.getHostAddress()).thenReturn("127.0.0.1"); - final InstanceConnectionInfo info = new InstanceConnectionInfo(address, 19871); + final TaskManagerLocation info = new TaskManagerLocation(ResourceID.generate(), address, 19871); Assert.assertEquals("worker2", info.getHostname()); } catch (Exception e) { @@ -163,7 +164,7 @@ public void testGetHostname1() { when(address.getHostName()).thenReturn("worker10"); when(address.getHostAddress()).thenReturn("127.0.0.1"); - InstanceConnectionInfo info = new InstanceConnectionInfo(address, 19871); + TaskManagerLocation info = new TaskManagerLocation(ResourceID.generate(), address, 19871); Assert.assertEquals("worker10", info.getHostname()); } catch (Exception e) { @@ -184,7 +185,7 @@ public void testGetHostname2() { when(address.getHostAddress()).thenReturn("192.168.254.254"); when(address.getAddress()).thenReturn(new byte[] {(byte) 192, (byte) 168, (byte) 254, (byte) 254} ); - InstanceConnectionInfo info = new InstanceConnectionInfo(address, 54152); + TaskManagerLocation info = new TaskManagerLocation(ResourceID.generate(), address, 54152); assertNotNull(info.getFQDNHostname()); assertTrue(info.getFQDNHostname().equals(addressString)); diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerRegistrationTest.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerRegistrationTest.scala index 7feb9493d27a4..f9c9b63ca6682 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerRegistrationTest.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerRegistrationTest.scala @@ -30,6 +30,7 @@ import org.apache.flink.runtime.instance._ import org.apache.flink.runtime.jobmanager.JobManagerRegistrationTest.PlainForwardingActor import org.apache.flink.runtime.messages.JobManagerMessages.LeaderSessionMessage import org.apache.flink.runtime.messages.RegistrationMessages.{AcknowledgeRegistration, AlreadyRegistered, RegisterTaskManager} +import org.apache.flink.runtime.taskmanager.TaskManagerLocation import org.apache.flink.runtime.testutils.TestingResourceManager import org.apache.flink.runtime.util.LeaderRetrievalUtils @@ -63,8 +64,11 @@ ImplicitSender with WordSpecLike with Matchers with BeforeAndAfterAll { val tm1 = _system.actorOf(Props(new PlainForwardingActor(testActor))) val tm2 = _system.actorOf(Props(new PlainForwardingActor(testActor))) - val connectionInfo1 = new InstanceConnectionInfo(InetAddress.getLocalHost, 10000) - val connectionInfo2 = new InstanceConnectionInfo(InetAddress.getLocalHost, 10001) + val resourceId1 = ResourceID.generate() + val resourceId2 = ResourceID.generate() + + val connectionInfo1 = new TaskManagerLocation(resourceId1, InetAddress.getLocalHost, 10000) + val connectionInfo2 = new TaskManagerLocation(resourceId2, InetAddress.getLocalHost, 10001) val hardwareDescription = HardwareDescription.extractFromSystem(10) @@ -75,7 +79,7 @@ ImplicitSender with WordSpecLike with Matchers with BeforeAndAfterAll { within(10 seconds) { jm.tell( RegisterTaskManager( - ResourceID.generate(), + resourceId1, connectionInfo1, hardwareDescription, 1), @@ -92,7 +96,7 @@ ImplicitSender with WordSpecLike with Matchers with BeforeAndAfterAll { within(10 seconds) { jm.tell( RegisterTaskManager( - ResourceID.generate(), + resourceId2, connectionInfo2, hardwareDescription, 1), @@ -118,7 +122,7 @@ ImplicitSender with WordSpecLike with Matchers with BeforeAndAfterAll { val selfGateway = new AkkaActorGateway(testActor, null) val resourceID = ResourceID.generate() - val connectionInfo = new InstanceConnectionInfo(InetAddress.getLocalHost,1) + val connectionInfo = new TaskManagerLocation(resourceID, InetAddress.getLocalHost, 1) val hardwareDescription = HardwareDescription.extractFromSystem(10) within(20 seconds) { diff --git a/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala b/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala index 73ab7ebf39c1c..f9f294b91d017 100644 --- a/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala +++ b/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala @@ -19,12 +19,11 @@ package org.apache.flink.yarn import org.apache.flink.runtime.clusterframework.types.ResourceID -import org.apache.flink.runtime.instance.InstanceConnectionInfo import org.apache.flink.runtime.io.disk.iomanager.IOManager import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService import org.apache.flink.runtime.memory.MemoryManager -import org.apache.flink.runtime.taskmanager.TaskManagerConfiguration +import org.apache.flink.runtime.taskmanager.{TaskManagerLocation, TaskManagerConfiguration} import org.apache.flink.runtime.testingUtils.TestingTaskManagerLike /** [[YarnTaskManager]] implementation which mixes in the [[TestingTaskManagerLike]] mixin. @@ -43,14 +42,14 @@ import org.apache.flink.runtime.testingUtils.TestingTaskManagerLike * JobManager */ class TestingYarnTaskManager( - config: TaskManagerConfiguration, - resourceID: ResourceID, - connectionInfo: InstanceConnectionInfo, - memoryManager: MemoryManager, - ioManager: IOManager, - network: NetworkEnvironment, - numberOfSlots: Int, - leaderRetrievalService: LeaderRetrievalService) + config: TaskManagerConfiguration, + resourceID: ResourceID, + connectionInfo: TaskManagerLocation, + memoryManager: MemoryManager, + ioManager: IOManager, + network: NetworkEnvironment, + numberOfSlots: Int, + leaderRetrievalService: LeaderRetrievalService) extends YarnTaskManager( config, resourceID, @@ -65,6 +64,7 @@ class TestingYarnTaskManager( object YarnTaskManager { /** Entry point (main method) to run the TaskManager on YARN. + * * @param args The command line arguments. */ def main(args: Array[String]): Unit = { diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala index 107801dd6c2df..0c6264ed7229d 100644 --- a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala +++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala @@ -19,12 +19,11 @@ package org.apache.flink.yarn import org.apache.flink.runtime.clusterframework.types.ResourceID -import org.apache.flink.runtime.instance.InstanceConnectionInfo import org.apache.flink.runtime.io.disk.iomanager.IOManager import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService import org.apache.flink.runtime.memory.MemoryManager -import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerConfiguration} +import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerConfiguration, TaskManagerLocation} /** An extension of the TaskManager that listens for additional YARN related * messages. @@ -32,7 +31,7 @@ import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerConfigurati class YarnTaskManager( config: TaskManagerConfiguration, resourceID: ResourceID, - connectionInfo: InstanceConnectionInfo, + taskManagerLocation: TaskManagerLocation, memoryManager: MemoryManager, ioManager: IOManager, network: NetworkEnvironment, @@ -41,7 +40,7 @@ class YarnTaskManager( extends TaskManager( config, resourceID, - connectionInfo, + taskManagerLocation, memoryManager, ioManager, network, From f214106cb1b3a525d3c33db0835722c3c39a81df Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 30 Aug 2016 20:34:20 +0200 Subject: [PATCH 017/299] [FLINK-4490] [distributed coordination] (part 2) Make slots independent of 'Instance'. To allow for a future dynamic slot allocation and release model, the slots should not depend on 'Instance'. In this change, the Slots hold most of the necessary information directly (location, gateway) and the interact with the Instance only via a 'SlotOwner' interface. --- .../InputChannelDeploymentDescriptor.java | 14 +- .../runtime/executiongraph/Execution.java | 51 +++---- .../executiongraph/ExecutionVertex.java | 17 +-- .../runtime/instance/HardwareDescription.java | 28 ++-- .../flink/runtime/instance/Instance.java | 22 +-- .../flink/runtime/instance/SharedSlot.java | 45 ++++-- .../flink/runtime/instance/SimpleSlot.java | 35 +++-- .../apache/flink/runtime/instance/Slot.java | 103 ++++++++++--- .../instance/SlotSharingGroupAssignment.java | 132 +++++++--------- .../scheduler/CoLocationConstraint.java | 48 +++--- .../jobmanager/scheduler/Scheduler.java | 74 +++++---- .../scheduler/SlotAllocationFuture.java | 116 ++++++++++---- .../runtime/jobmanager/slots/SlotOwner.java | 29 ++++ .../taskmanager/TaskManagerLocation.java | 2 +- .../flink/runtime/jobmanager/JobManager.scala | 2 +- .../testingUtils/TestingJobManagerLike.scala | 2 +- .../ExecutionGraphMetricsTest.java | 4 +- .../VertexLocationConstraintTest.java | 52 +++---- .../runtime/instance/SharedSlotsTest.java | 67 ++++---- .../ScheduleWithCoLocationHintTest.java | 144 ++++++++++-------- .../scheduler/SchedulerIsolatedTasksTest.java | 52 ++++--- .../scheduler/SchedulerSlotSharingTest.java | 102 +++++++------ .../scheduler/SchedulerTestUtils.java | 29 +++- .../scheduler/SlotAllocationFutureTest.java | 51 +++++-- .../ResourceManagerITCase.java | 17 ++- 25 files changed, 741 insertions(+), 497 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/SlotOwner.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptor.java index f31febb875c22..0912055b48815 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/InputChannelDeploymentDescriptor.java @@ -18,16 +18,18 @@ package org.apache.flink.runtime.deployment; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.Execution; import org.apache.flink.runtime.executiongraph.ExecutionEdge; import org.apache.flink.runtime.executiongraph.IntermediateResultPartition; -import org.apache.flink.runtime.instance.Instance; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.io.network.ConnectionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.consumer.InputChannel; import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -88,6 +90,7 @@ public String toString() { public static InputChannelDeploymentDescriptor[] fromEdges( ExecutionEdge[] edges, SimpleSlot consumerSlot) { + final ResourceID consumerTaskManager = consumerSlot.getTaskManagerID(); final InputChannelDeploymentDescriptor[] icdd = new InputChannelDeploymentDescriptor[edges.length]; // Each edge is connected to a different result partition @@ -105,16 +108,17 @@ public static InputChannelDeploymentDescriptor[] fromEdges( (producerState == ExecutionState.RUNNING || producerState == ExecutionState.FINISHED)) { - final Instance partitionInstance = producerSlot.getInstance(); + final TaskManagerLocation partitionTaskManagerLocation = producerSlot.getTaskManagerLocation(); + final ResourceID partitionTaskManager = partitionTaskManagerLocation.getResourceID(); - if (partitionInstance.equals(consumerSlot.getInstance())) { - // Consuming task is deployed to the same instance as the partition => local + if (partitionTaskManager.equals(consumerTaskManager)) { + // Consuming task is deployed to the same TaskManager as the partition => local partitionLocation = ResultPartitionLocation.createLocal(); } else { // Different instances => remote final ConnectionID connectionId = new ConnectionID( - partitionInstance.getInstanceConnectionInfo(), + partitionTaskManagerLocation, consumedPartition.getIntermediateResult().getConnectionIndex()); partitionLocation = ResultPartitionLocation.createRemote(connectionId); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java index 197999ce37c92..846df4956d854 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java @@ -25,12 +25,12 @@ import org.apache.flink.runtime.JobException; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor; import org.apache.flink.runtime.deployment.PartialInputChannelDeploymentDescriptor; import org.apache.flink.runtime.deployment.ResultPartitionLocation; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.execution.ExecutionState; -import org.apache.flink.runtime.instance.Instance; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.instance.SimpleSlot; @@ -49,7 +49,6 @@ import org.apache.flink.runtime.state.ChainedStateHandle; import org.apache.flink.runtime.state.KeyGroupsStateHandle; import org.apache.flink.runtime.state.StreamStateHandle; -import org.apache.flink.runtime.util.SerializableObject; import org.apache.flink.util.ExceptionUtils; import org.slf4j.Logger; @@ -371,7 +370,7 @@ public void deployToSlot(final SimpleSlot slot) throws JobException { throw new JobException("Could not assign the ExecutionVertex to the slot " + slot); } this.assignedResource = slot; - this.assignedResourceLocation = slot.getInstance().getInstanceConnectionInfo(); + this.assignedResourceLocation = slot.getTaskManagerLocation(); // race double check, did we fail/cancel and do we need to release the slot? if (this.state != DEPLOYING) { @@ -381,7 +380,7 @@ public void deployToSlot(final SimpleSlot slot) throws JobException { if (LOG.isInfoEnabled()) { LOG.info(String.format("Deploying %s (attempt #%d) to %s", vertex.getSimpleName(), - attemptNumber, slot.getInstance().getInstanceConnectionInfo().getHostname())); + attemptNumber, assignedResourceLocation.getHostname())); } final TaskDeploymentDescriptor deployment = vertex.createDeploymentDescriptor( @@ -393,9 +392,8 @@ public void deployToSlot(final SimpleSlot slot) throws JobException { // register this execution at the execution graph, to receive call backs vertex.getExecutionGraph().registerExecution(this); - - final Instance instance = slot.getInstance(); - final ActorGateway gateway = instance.getActorGateway(); + + final ActorGateway gateway = slot.getTaskManagerActorGateway(); final Future deployAction = gateway.ask(new SubmitTask(deployment), timeout); @@ -408,7 +406,7 @@ public void onComplete(Throwable failure, Object success) throws Throwable { String taskname = deployment.getTaskInfo().getTaskNameWithSubtasks() + " (" + attemptId + ')'; markFailed(new Exception( - "Cannot deploy task " + taskname + " - TaskManager (" + instance + "Cannot deploy task " + taskname + " - TaskManager (" + assignedResourceLocation + ") not responding after a timeout of " + timeout, failure)); } else { @@ -437,7 +435,7 @@ public void stop() { final SimpleSlot slot = this.assignedResource; if (slot != null) { - final ActorGateway gateway = slot.getInstance().getActorGateway(); + final ActorGateway gateway = slot.getTaskManagerActorGateway(); Future stopResult = gateway.retry( new StopTask(attemptId), @@ -590,24 +588,25 @@ public Boolean call() throws Exception { continue; } - final Instance consumerInstance = consumerSlot.getInstance(); - - final ResultPartitionID partitionId = new ResultPartitionID( - partition.getPartitionId(), attemptId); + final TaskManagerLocation partitionTaskManagerLocation = partition.getProducer() + .getCurrentAssignedResource().getTaskManagerLocation(); + final ResourceID partitionTaskManager = partitionTaskManagerLocation.getResourceID(); + + final ResourceID consumerTaskManager = consumerSlot.getTaskManagerID(); - final Instance partitionInstance = partition.getProducer() - .getCurrentAssignedResource().getInstance(); + final ResultPartitionID partitionId = new ResultPartitionID(partition.getPartitionId(), attemptId); + final ResultPartitionLocation partitionLocation; - if (consumerInstance.equals(partitionInstance)) { + if (consumerTaskManager.equals(partitionTaskManager)) { // Consuming task is deployed to the same instance as the partition => local partitionLocation = ResultPartitionLocation.createLocal(); } else { // Different instances => remote final ConnectionID connectionId = new ConnectionID( - partitionInstance.getInstanceConnectionInfo(), + partitionTaskManagerLocation, partition.getIntermediateResult().getConnectionIndex()); partitionLocation = ResultPartitionLocation.createRemote(connectionId); @@ -916,7 +915,7 @@ private void sendCancelRpcCall() { if (slot != null) { - final ActorGateway gateway = slot.getInstance().getActorGateway(); + final ActorGateway gateway = slot.getTaskManagerActorGateway(); Future cancelResult = gateway.retry( new CancelTask(attemptId), @@ -946,14 +945,10 @@ private void sendFailIntermediateResultPartitionsRpcCall() { final SimpleSlot slot = this.assignedResource; if (slot != null) { - final Instance instance = slot.getInstance(); + final ActorGateway gateway = slot.getTaskManagerActorGateway(); - if (instance.isAlive()) { - final ActorGateway gateway = instance.getActorGateway(); - - // TODO For some tests this could be a problem when querying too early if all resources were released - gateway.tell(new FailIntermediateResultPartitions(attemptId)); - } + // TODO For some tests this could be a problem when querying too early if all resources were released + gateway.tell(new FailIntermediateResultPartitions(attemptId)); } } @@ -968,15 +963,15 @@ private void sendUpdatePartitionInfoRpcCall( final UpdatePartitionInfo updatePartitionInfo) { if (consumerSlot != null) { - final Instance instance = consumerSlot.getInstance(); - final ActorGateway gateway = instance.getActorGateway(); + final ActorGateway gateway = consumerSlot.getTaskManagerActorGateway(); + final TaskManagerLocation taskManagerLocation = consumerSlot.getTaskManagerLocation(); Future futureUpdate = gateway.ask(updatePartitionInfo, timeout); futureUpdate.onFailure(new OnFailure() { @Override public void onFailure(Throwable failure) throws Throwable { - fail(new IllegalStateException("Update task on instance " + instance + + fail(new IllegalStateException("Update task on TaskManager " + taskManagerLocation + " failed due to:", failure)); } }, executionContext); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java index e5a115a6eed69..f02647e001ade 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java @@ -27,7 +27,6 @@ import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.execution.ExecutionState; -import org.apache.flink.runtime.instance.Instance; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.instance.SimpleSlot; @@ -98,7 +97,7 @@ public class ExecutionVertex { private volatile Execution currentExecution; // this field must never be null - private volatile List locationConstraintInstances; + private volatile List locationConstraintInstances; private volatile boolean scheduleLocalOnly; @@ -352,7 +351,7 @@ else if (numSources < parallelism) { } } - public void setLocationConstraintHosts(List instances) { + public void setLocationConstraintHosts(List instances) { this.locationConstraintInstances = instances; } @@ -376,9 +375,9 @@ public boolean isScheduleLocalOnly() { * * @return The preferred locations for this vertex execution, or null, if there is no preference. */ - public Iterable getPreferredLocations() { + public Iterable getPreferredLocations() { // if we have hard location constraints, use those - List constraintInstances = this.locationConstraintInstances; + List constraintInstances = this.locationConstraintInstances; if (constraintInstances != null && !constraintInstances.isEmpty()) { return constraintInstances; } @@ -388,8 +387,8 @@ public Iterable getPreferredLocations() { return Collections.emptySet(); } else { - Set locations = new HashSet(); - Set inputLocations = new HashSet(); + Set locations = new HashSet<>(); + Set inputLocations = new HashSet<>(); // go over all inputs for (int i = 0; i < inputEdges.length; i++) { @@ -402,7 +401,7 @@ public Iterable getPreferredLocations() { SimpleSlot sourceSlot = sources[k].getSource().getProducer().getCurrentAssignedResource(); if (sourceSlot != null) { // add input location - inputLocations.add(sourceSlot.getInstance()); + inputLocations.add(sourceSlot.getTaskManagerLocation()); // inputs which have too many distinct sources are not considered if (inputLocations.size() > MAX_DISTINCT_LOCATIONS_TO_CONSIDER) { inputLocations.clear(); @@ -495,7 +494,7 @@ public boolean sendMessageToCurrentExecution( // send only if we actually have a target if (slot != null) { - ActorGateway gateway = slot.getInstance().getActorGateway(); + ActorGateway gateway = slot.getTaskManagerActorGateway(); if (gateway != null) { if (sender == null) { gateway.tell(message); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/HardwareDescription.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/HardwareDescription.java index bfcc1e5c141a4..9c1c5b78a45f6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/HardwareDescription.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/HardwareDescription.java @@ -30,22 +30,16 @@ public final class HardwareDescription implements Serializable { private static final long serialVersionUID = 3380016608300325361L; /** The number of CPU cores available to the JVM on the compute node. */ - private int numberOfCPUCores; + private final int numberOfCPUCores; /** The size of physical memory in bytes available on the compute node. */ - private long sizeOfPhysicalMemory; + private final long sizeOfPhysicalMemory; /** The size of the JVM heap memory */ - private long sizeOfJvmHeap; - - /** The size of the memory managed by the system for caching, hashing, sorting, ... */ - private long sizeOfManagedMemory; + private final long sizeOfJvmHeap; - - /** - * Public default constructor used for serialization process. - */ - public HardwareDescription() {} + /** The size of the memory managed by the system for caching, hashing, sorting, ... */ + private final long sizeOfManagedMemory; /** * Constructs a new hardware description object. @@ -88,7 +82,7 @@ public long getSizeOfPhysicalMemory() { public long getSizeOfJvmHeap() { return this.sizeOfJvmHeap; } - + /** * Returns the size of the memory managed by the system for caching, hashing, sorting, ... * @@ -97,26 +91,26 @@ public long getSizeOfJvmHeap() { public long getSizeOfManagedMemory() { return this.sizeOfManagedMemory; } - + // -------------------------------------------------------------------------------------------- // Utils // -------------------------------------------------------------------------------------------- - + @Override public String toString() { return String.format("cores=%d, physMem=%d, heap=%d, managed=%d", numberOfCPUCores, sizeOfPhysicalMemory, sizeOfJvmHeap, sizeOfManagedMemory); } - + // -------------------------------------------------------------------------------------------- // Factory // -------------------------------------------------------------------------------------------- - + public static HardwareDescription extractFromSystem(long managedMemory) { final int numberOfCPUCores = Hardware.getNumberCPUCores(); final long sizeOfJvmHeap = Runtime.getRuntime().maxMemory(); final long sizeOfPhysicalMemory = Hardware.getSizeOfPhysicalMemory(); - + return new HardwareDescription(numberOfCPUCores, sizeOfPhysicalMemory, sizeOfJvmHeap, managedMemory); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java index 598b32bbc89ac..fe46895c4500b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java @@ -28,15 +28,20 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.jobmanager.scheduler.SlotAvailabilityListener; +import org.apache.flink.runtime.jobmanager.slots.SlotOwner; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkArgument; + /** * An instance represents a {@link org.apache.flink.runtime.taskmanager.TaskManager} * registered at a JobManager and ready to receive work. */ -public class Instance { +public class Instance implements SlotOwner { private final static Logger LOG = LoggerFactory.getLogger(Instance.class); @@ -241,7 +246,7 @@ public SimpleSlot allocateSimpleSlot(JobID jobID) throws InstanceDiedException { return null; } else { - SimpleSlot slot = new SimpleSlot(jobID, this, nextSlot); + SimpleSlot slot = new SimpleSlot(jobID, this, connectionInfo, nextSlot, actorGateway); allocatedSlots.add(slot); return slot; } @@ -278,7 +283,8 @@ public SharedSlot allocateSharedSlot(JobID jobID, SlotSharingGroupAssignment sha return null; } else { - SharedSlot slot = new SharedSlot(jobID, this, nextSlot, sharingGroupAssignment); + SharedSlot slot = new SharedSlot( + jobID, this, connectionInfo, nextSlot, actorGateway, sharingGroupAssignment); allocatedSlots.add(slot); return slot; } @@ -295,13 +301,11 @@ public SharedSlot allocateSharedSlot(JobID jobID, SlotSharingGroupAssignment sha * @param slot The slot to return. * @return True, if the slot was returned, false if not. */ + @Override public boolean returnAllocatedSlot(Slot slot) { - if (slot == null || slot.getInstance() != this) { - throw new IllegalArgumentException("Slot is null or belongs to the wrong TaskManager."); - } - if (slot.isAlive()) { - throw new IllegalArgumentException("Slot is still alive"); - } + checkNotNull(slot); + checkArgument(!slot.isAlive(), "slot is still alive"); + checkArgument(slot.getOwner() == this, "slot belongs to the wrong TaskManager."); if (slot.markReleased()) { LOG.debug("Return allocated slot {}.", slot); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SharedSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SharedSlot.java index ef6291069ced2..7f056042b9098 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SharedSlot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SharedSlot.java @@ -18,13 +18,18 @@ package org.apache.flink.runtime.instance; +import org.apache.flink.runtime.jobmanager.slots.SlotOwner; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.AbstractID; import org.apache.flink.api.common.JobID; +import javax.annotation.Nullable; import java.util.ConcurrentModificationException; import java.util.HashSet; import java.util.Set; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * This class represents a shared slot. A shared slot can have multiple * {@link SimpleSlot} instances within itself. This allows to @@ -35,7 +40,7 @@ *

IMPORTANT: This class contains no synchronization. Thus, the caller has to guarantee proper * synchronization. In the current implementation, all concurrently modifying operations are * passed through a {@link SlotSharingGroupAssignment} object which is responsible for - * synchronization.

+ * synchronization. */ public class SharedSlot extends Slot { @@ -51,12 +56,18 @@ public class SharedSlot extends Slot { * This constructor is used to create a slot directly from an instance. * * @param jobID The ID of the job that the slot is created for. - * @param instance The instance that holds the slot. + * @param owner The component from which this slot is allocated. + * @param location The location info of the TaskManager where the slot was allocated from * @param slotNumber The number of the slot. + * @param taskManagerActorGateway The actor gateway to communicate with the TaskManager * @param assignmentGroup The assignment group that this shared slot belongs to. */ - public SharedSlot(JobID jobID, Instance instance, int slotNumber, SlotSharingGroupAssignment assignmentGroup) { - this(jobID, instance, slotNumber, assignmentGroup, null, null); + public SharedSlot( + JobID jobID, SlotOwner owner, TaskManagerLocation location, int slotNumber, + ActorGateway taskManagerActorGateway, + SlotSharingGroupAssignment assignmentGroup) { + + this(jobID, owner, location, slotNumber, taskManagerActorGateway, assignmentGroup, null, null); } /** @@ -64,15 +75,23 @@ public SharedSlot(JobID jobID, Instance instance, int slotNumber, SlotSharingGro * to the given task group. * * @param jobID The ID of the job that the slot is created for. - * @param instance The instance that holds the slot. + * @param owner The component from which this slot is allocated. + * @param location The location info of the TaskManager where the slot was allocated from * @param slotNumber The number of the slot. + * @param taskManagerActorGateway The actor gateway to communicate with the TaskManager * @param assignmentGroup The assignment group that this shared slot belongs to. + * @param parent The parent slot of this slot. + * @param groupId The assignment group of this slot. */ - public SharedSlot(JobID jobID, Instance instance, int slotNumber, - SlotSharingGroupAssignment assignmentGroup, SharedSlot parent, AbstractID groupId) { - super(jobID, instance, slotNumber, parent, groupId); + public SharedSlot( + JobID jobID, SlotOwner owner, TaskManagerLocation location, int slotNumber, + ActorGateway taskManagerActorGateway, + SlotSharingGroupAssignment assignmentGroup, + @Nullable SharedSlot parent, @Nullable AbstractID groupId) { + + super(jobID, owner, location, slotNumber, taskManagerActorGateway, parent, groupId); - this.assignmentGroup = assignmentGroup; + this.assignmentGroup = checkNotNull(assignmentGroup); this.subSlots = new HashSet(); } @@ -148,7 +167,9 @@ Set getSubSlots() { */ SimpleSlot allocateSubSlot(AbstractID groupId) { if (isAlive()) { - SimpleSlot slot = new SimpleSlot(getJobID(), getInstance(), subSlots.size(), this, groupId); + SimpleSlot slot = new SimpleSlot( + getJobID(), getOwner(), getTaskManagerLocation(), subSlots.size(), + getTaskManagerActorGateway(), this, groupId); subSlots.add(slot); return slot; } @@ -168,7 +189,9 @@ SimpleSlot allocateSubSlot(AbstractID groupId) { */ SharedSlot allocateSharedSlot(AbstractID groupId){ if (isAlive()) { - SharedSlot slot = new SharedSlot(getJobID(), getInstance(), subSlots.size(), assignmentGroup, this, groupId); + SharedSlot slot = new SharedSlot( + getJobID(), getOwner(), getTaskManagerLocation(), subSlots.size(), + getTaskManagerActorGateway(), assignmentGroup, this, groupId); subSlots.add(slot); return slot; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SimpleSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SimpleSlot.java index dbe961ae1d706..615138f42698b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SimpleSlot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SimpleSlot.java @@ -21,15 +21,18 @@ import org.apache.flink.runtime.executiongraph.Execution; import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.jobmanager.scheduler.Locality; +import org.apache.flink.runtime.jobmanager.slots.SlotOwner; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.AbstractID; +import javax.annotation.Nullable; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; /** * A SimpleSlot represents a single slot on a TaskManager instance, or a slot within a shared slot. * *

If this slot is part of a {@link SharedSlot}, then the parent attribute will point to that shared slot. - * If not, then the parent attribute is null.

+ * If not, then the parent attribute is null. */ public class SimpleSlot extends Slot { @@ -43,18 +46,22 @@ public class SimpleSlot extends Slot { private volatile Execution executedTask; /** The locality attached to the slot, defining whether the slot was allocated at the desired location. */ - private Locality locality = Locality.UNCONSTRAINED; + private volatile Locality locality = Locality.UNCONSTRAINED; /** * Creates a new simple slot that stands alone and does not belong to shared slot. * * @param jobID The ID of the job that the slot is allocated for. - * @param instance The instance that the slot belongs to. + * @param owner The component from which this slot is allocated. + * @param location The location info of the TaskManager where the slot was allocated from * @param slotNumber The number of the task slot on the instance. + * @param taskManagerActorGateway The actor gateway to communicate with the TaskManager of this slot */ - public SimpleSlot(JobID jobID, Instance instance, int slotNumber) { - super(jobID, instance, slotNumber, null, null); + public SimpleSlot( + JobID jobID, SlotOwner owner, TaskManagerLocation location, int slotNumber, + ActorGateway taskManagerActorGateway) { + this(jobID, owner, location, slotNumber, taskManagerActorGateway, null, null); } /** @@ -62,13 +69,18 @@ public SimpleSlot(JobID jobID, Instance instance, int slotNumber) { * is identified by the given ID.. * * @param jobID The ID of the job that the slot is allocated for. - * @param instance The instance that the slot belongs to. + * @param owner The component from which this slot is allocated. + * @param location The location info of the TaskManager where the slot was allocated from * @param slotNumber The number of the simple slot in its parent shared slot. * @param parent The parent shared slot. * @param groupID The ID that identifies the group that the slot belongs to. */ - public SimpleSlot(JobID jobID, Instance instance, int slotNumber, SharedSlot parent, AbstractID groupID) { - super(jobID, instance, slotNumber, parent, groupID); + public SimpleSlot( + JobID jobID, SlotOwner owner, TaskManagerLocation location, int slotNumber, + ActorGateway taskManagerActorGateway, + @Nullable SharedSlot parent, @Nullable AbstractID groupID) { + + super(jobID, owner, location, slotNumber, taskManagerActorGateway, parent, groupID); } // ------------------------------------------------------------------------ @@ -142,15 +154,12 @@ public void setLocality(Locality locality) { @Override public void releaseSlot() { - if (!isCanceled()) { // kill all tasks currently running in this slot Execution exec = this.executedTask; if (exec != null && !exec.isFinished()) { - exec.fail(new Exception( - "The slot in which the task was executed has been released. Probably loss of TaskManager " - + getInstance())); + exec.fail(new Exception("TaskManager was lost/killed: " + getTaskManagerLocation())); } // release directly (if we are directly allocated), @@ -158,7 +167,7 @@ public void releaseSlot() { if (getParent() == null) { // we have to give back the slot to the owning instance if (markCancelled()) { - getInstance().returnAllocatedSlot(this); + getOwner().returnAllocatedSlot(this); } } else { // we have to ask our parent to dispose us diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Slot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Slot.java index 341ef95481681..451a9ecbed595 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Slot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Slot.java @@ -18,11 +18,18 @@ package org.apache.flink.runtime.instance; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.jobmanager.slots.SlotOwner; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.AbstractID; import org.apache.flink.api.common.JobID; +import javax.annotation.Nullable; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * Base class for task slots. TaskManagers offer one or more task slots, which define a slice of * their resources. @@ -30,7 +37,7 @@ *

In the simplest case, a slot holds a single task ({@link SimpleSlot}). In the more complex * case, a slot is shared ({@link SharedSlot}) and contains a set of tasks. Shared slots may contain * other shared slots which in turn can hold simple slots. That way, a shared slot may define a tree - * of slots that belong to it.

+ * of slots that belong to it. */ public abstract class Slot { @@ -52,15 +59,23 @@ public abstract class Slot { /** The ID of the job this slice belongs to. */ private final JobID jobID; - /** The id of the group that this slot is allocated to. May be null. */ - private final AbstractID groupID; + /** The location information of the TaskManager to which this slot belongs */ + private final TaskManagerLocation taskManagerLocation; + + /** TEMP until the new RPC is in place: The actor gateway to communicate with the TaskManager */ + private final ActorGateway taskManagerActorGateway; - /** The instance on which the slot is allocated */ - private final Instance instance; + /** The owner of this slot - the slot was taken from that owner and must be disposed to it */ + private final SlotOwner owner; /** The parent of this slot in the hierarchy, or null, if this is the parent */ + @Nullable private final SharedSlot parent; + /** The id of the group that this slot is allocated to. May be null. */ + @Nullable + private final AbstractID groupID; + /** The number of the slot on which the task is deployed */ private final int slotNumber; @@ -71,23 +86,28 @@ public abstract class Slot { * Base constructor for slots. * * @param jobID The ID of the job that this slot is allocated for. - * @param instance The instance from which this slot is allocated. + * @param owner The component from which this slot is allocated. + * @param location The location info of the TaskManager where the slot was allocated from * @param slotNumber The number of this slot. + * @param taskManagerActorGateway The actor gateway to communicate with the TaskManager * @param parent The parent slot that contains this slot. May be null, if this slot is the root. * @param groupID The ID that identifies the task group for which this slot is allocated. May be null * if the slot does not belong to any task group. */ - protected Slot(JobID jobID, Instance instance, int slotNumber, SharedSlot parent, AbstractID groupID) { - if (jobID == null || instance == null || slotNumber < 0) { - throw new IllegalArgumentException(); - } - - this.jobID = jobID; - this.instance = instance; + protected Slot( + JobID jobID, SlotOwner owner, TaskManagerLocation location, int slotNumber, + ActorGateway taskManagerActorGateway, + @Nullable SharedSlot parent, @Nullable AbstractID groupID) { + + checkArgument(slotNumber >= 0); + + this.jobID = checkNotNull(jobID); + this.taskManagerLocation = checkNotNull(location); + this.owner = checkNotNull(owner); + this.taskManagerActorGateway = checkNotNull(taskManagerActorGateway); + this.parent = parent; // may be null + this.groupID = groupID; // may be null this.slotNumber = slotNumber; - this.parent = parent; - this.groupID = groupID; - } // -------------------------------------------------------------------------------------------- @@ -101,12 +121,42 @@ public JobID getJobID() { } /** - * Gets the instance from which the slot was allocated. + * Gets the ID of the TaskManager that offers this slot. + * + * @return The ID of the TaskManager that offers this slot + */ + public ResourceID getTaskManagerID() { + return taskManagerLocation.getResourceID(); + } + + /** + * Gets the location info of the TaskManager that offers this slot. * - * @return The instance from which the slot was allocated. + * @return The location info of the TaskManager that offers this slot */ - public Instance getInstance() { - return instance; + public TaskManagerLocation getTaskManagerLocation() { + return taskManagerLocation; + } + + /** + * Gets the actor gateway that can be used to send messages to the TaskManager. + * + *

This method should be removed once the new interface-based RPC abstraction is in place + * + * @return The actor gateway that can be used to send messages to the TaskManager. + */ + public ActorGateway getTaskManagerActorGateway() { + return taskManagerActorGateway; + } + + /** + * Gets the owner of this slot. The owner is the component that the slot was created from + * and to which it needs to be returned after the executed tasks are done. + * + * @return The owner of this slot. + */ + public SlotOwner getOwner() { + return owner; } /** @@ -149,6 +199,7 @@ public int getRootSlotNumber() { * * @return The ID identifying the logical group of slots. */ + @Nullable public AbstractID getGroupID() { return groupID; } @@ -158,10 +209,18 @@ public AbstractID getGroupID() { * * @return The parent slot, or null, if no this slot has no parent. */ + @Nullable public SharedSlot getParent() { return parent; } + /** + * Gets the root slot of the tree containing this slot. If this slot is the root, + * the method returns this slot directly, otherwise it recursively goes to the parent until + * it reaches the root. + * + * @return The root slot of the tree containing this slot + */ public Slot getRoot() { if (parent == null) { return this; @@ -244,11 +303,11 @@ final boolean markReleased() { @Override public String toString() { - return hierarchy() + " - " + instance + " - " + getStateName(status); + return hierarchy() + " - " + taskManagerLocation + " - " + getStateName(status); } protected String hierarchy() { - return (getParent() != null ? getParent().hierarchy() : "") + "(" + slotNumber + ")"; + return (getParent() != null ? getParent().hierarchy() : "") + '(' + slotNumber + ')'; } private static String getStateName(int state) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignment.java index 7d666fe09dbf5..346cc77b46937 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignment.java @@ -28,13 +28,15 @@ import java.util.Map; import java.util.Set; -import org.apache.commons.lang3.tuple.ImmutablePair; -import org.apache.commons.lang3.tuple.Pair; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.jobmanager.scheduler.CoLocationConstraint; import org.apache.flink.runtime.jobmanager.scheduler.Locality; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.AbstractID; import org.apache.flink.runtime.executiongraph.ExecutionVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -93,9 +95,8 @@ public class SlotSharingGroupAssignment { /** All slots currently allocated to this sharing group */ private final Set allSlots = new LinkedHashSet(); - /** The slots available per vertex type (jid), keyed by instance, to make them locatable */ - private final Map>> availableSlotsPerJid = - new LinkedHashMap>>(); + /** The slots available per vertex type (JobVertexId), keyed by TaskManager, to make them locatable */ + private final Map>> availableSlotsPerJid = new LinkedHashMap<>(); // -------------------------------------------------------------------------------------------- @@ -122,7 +123,7 @@ public int getNumberOfSlots() { */ public int getNumberOfAvailableSlotsForGroup(AbstractID groupId) { synchronized (lock) { - Map> available = availableSlotsPerJid.get(groupId); + Map> available = availableSlotsPerJid.get(groupId); if (available != null) { Set set = new HashSet(); @@ -148,37 +149,25 @@ public int getNumberOfAvailableSlotsForGroup(AbstractID groupId) { // Slot allocation // ------------------------------------------------------------------------ - /** - * - * @param sharedSlot - * @param locality - * @param groupId - * @return - */ public SimpleSlot addSharedSlotAndAllocateSubSlot(SharedSlot sharedSlot, Locality locality, JobVertexID groupId) { return addSharedSlotAndAllocateSubSlot(sharedSlot, locality, groupId, null); } - /** - * - * @param sharedSlot - * @param locality - * @param constraint - * @return - */ - public SimpleSlot addSharedSlotAndAllocateSubSlot(SharedSlot sharedSlot, Locality locality, - CoLocationConstraint constraint) { + public SimpleSlot addSharedSlotAndAllocateSubSlot( + SharedSlot sharedSlot, Locality locality, CoLocationConstraint constraint) + { return addSharedSlotAndAllocateSubSlot(sharedSlot, locality, null, constraint); } - - private SimpleSlot addSharedSlotAndAllocateSubSlot(SharedSlot sharedSlot, Locality locality, - JobVertexID groupId, CoLocationConstraint constraint) { + + private SimpleSlot addSharedSlotAndAllocateSubSlot( + SharedSlot sharedSlot, Locality locality, JobVertexID groupId, CoLocationConstraint constraint) { + // sanity checks if (!sharedSlot.isRootAndEmpty()) { throw new IllegalArgumentException("The given slot is not an empty root slot."); } - - final Instance location = sharedSlot.getInstance(); + + final ResourceID location = sharedSlot.getTaskManagerID(); synchronized (lock) { // early out in case that the slot died (instance disappeared) @@ -244,20 +233,20 @@ private SimpleSlot addSharedSlotAndAllocateSubSlot(SharedSlot sharedSlot, Locali // can place a task into this slot. boolean entryForNewJidExists = false; - for (Map.Entry>> entry : availableSlotsPerJid.entrySet()) { + for (Map.Entry>> entry : availableSlotsPerJid.entrySet()) { // there is already an entry for this groupID if (entry.getKey().equals(groupIdForMap)) { entryForNewJidExists = true; continue; } - Map> available = entry.getValue(); + Map> available = entry.getValue(); putIntoMultiMap(available, location, sharedSlot); } // make sure an empty entry exists for this group, if no other entry exists if (!entryForNewJidExists) { - availableSlotsPerJid.put(groupIdForMap, new LinkedHashMap>()); + availableSlotsPerJid.put(groupIdForMap, new LinkedHashMap>()); } return subSlot; @@ -287,18 +276,15 @@ public SimpleSlot getSlotForTask(ExecutionVertex vertex) { /** * - * @param vertexID - * @param locationPreferences - * @return */ - SimpleSlot getSlotForTask(JobVertexID vertexID, Iterable locationPreferences) { + SimpleSlot getSlotForTask(JobVertexID vertexID, Iterable locationPreferences) { synchronized (lock) { - Pair p = getSlotForTaskInternal(vertexID, locationPreferences, false); + Tuple2 p = getSlotForTaskInternal(vertexID, locationPreferences, false); if (p != null) { - SharedSlot ss = p.getLeft(); + SharedSlot ss = p.f0; SimpleSlot slot = ss.allocateSubSlot(vertexID); - slot.setLocality(p.getRight()); + slot.setLocality(p.f1); return slot; } else { @@ -330,7 +316,7 @@ public SimpleSlot getSlotForTask(ExecutionVertex vertex, CoLocationConstraint co return getSlotForTask(constraint, vertex.getPreferredLocations()); } - SimpleSlot getSlotForTask(CoLocationConstraint constraint, Iterable locationPreferences) { + SimpleSlot getSlotForTask(CoLocationConstraint constraint, Iterable locationPreferences) { synchronized (lock) { if (constraint.isAssignedAndAlive()) { // the shared slot of the co-location group is initialized and set we allocate a sub-slot @@ -346,15 +332,16 @@ else if (constraint.isAssigned()) { if (previous == null) { throw new IllegalStateException("Bug: Found assigned co-location constraint without a slot."); } - - Instance location = previous.getInstance(); - Pair p = getSlotForTaskInternal(constraint.getGroupId(), - Collections.singleton(location), true); + + TaskManagerLocation location = previous.getTaskManagerLocation(); + Tuple2 p = getSlotForTaskInternal( + constraint.getGroupId(), Collections.singleton(location), true); + if (p == null) { return null; } else { - SharedSlot newSharedSlot = p.getLeft(); + SharedSlot newSharedSlot = p.f0; // allocate the co-location group slot inside the shared slot SharedSlot constraintGroupSlot = newSharedSlot.allocateSharedSlot(constraint.getGroupId()); @@ -377,15 +364,15 @@ else if (constraint.isAssigned()) { // the location constraint has not been associated with a shared slot, yet. // grab a new slot and initialize the constraint with that one. // preferred locations are defined by the vertex - Pair p = + Tuple2 p = getSlotForTaskInternal(constraint.getGroupId(), locationPreferences, false); if (p == null) { // could not get a shared slot for this co-location-group return null; } else { - final SharedSlot availableShared = p.getLeft(); - final Locality l = p.getRight(); + final SharedSlot availableShared = p.f0; + final Locality l = p.f1; // allocate the co-location group slot inside the shared slot SharedSlot constraintGroupSlot = availableShared.allocateSharedSlot(constraint.getGroupId()); @@ -405,9 +392,8 @@ else if (constraint.isAssigned()) { } - private Pair getSlotForTaskInternal(AbstractID groupId, - Iterable preferredLocations, - boolean localOnly) + private Tuple2 getSlotForTaskInternal( + AbstractID groupId, Iterable preferredLocations, boolean localOnly) { // check if there is anything at all in this group assignment if (allSlots.isEmpty()) { @@ -415,15 +401,15 @@ private Pair getSlotForTaskInternal(AbstractID groupId, } // get the available slots for the group - Map> slotsForGroup = availableSlotsPerJid.get(groupId); + Map> slotsForGroup = availableSlotsPerJid.get(groupId); if (slotsForGroup == null) { // we have a new group, so all slots are available - slotsForGroup = new LinkedHashMap>(); + slotsForGroup = new LinkedHashMap<>(); availableSlotsPerJid.put(groupId, slotsForGroup); for (SharedSlot availableSlot : allSlots) { - putIntoMultiMap(slotsForGroup, availableSlot.getInstance(), availableSlot); + putIntoMultiMap(slotsForGroup, availableSlot.getTaskManagerID(), availableSlot); } } else if (slotsForGroup.isEmpty()) { @@ -435,15 +421,15 @@ else if (slotsForGroup.isEmpty()) { boolean didNotGetPreferred = false; if (preferredLocations != null) { - for (Instance location : preferredLocations) { + for (TaskManagerLocation location : preferredLocations) { // set the flag that we failed a preferred location. If one will be found, // we return early anyways and skip the flag evaluation didNotGetPreferred = true; - SharedSlot slot = removeFromMultiMap(slotsForGroup, location); + SharedSlot slot = removeFromMultiMap(slotsForGroup, location.getResourceID()); if (slot != null && slot.isAlive()) { - return new ImmutablePair(slot, Locality.LOCAL); + return new Tuple2<>(slot, Locality.LOCAL); } } } @@ -459,7 +445,7 @@ else if (slotsForGroup.isEmpty()) { SharedSlot slot; while ((slot = pollFromMultiMap(slotsForGroup)) != null) { if (slot.isAlive()) { - return new ImmutablePair(slot, locality); + return new Tuple2<>(slot, locality); } } @@ -510,7 +496,7 @@ void releaseSimpleSlot(SimpleSlot simpleSlot) { // for that group again. otherwise, the slot is part of a // co-location group and nothing becomes immediately available - Map> slotsForJid = availableSlotsPerJid.get(groupID); + Map> slotsForJid = availableSlotsPerJid.get(groupID); // sanity check if (slotsForJid == null) { @@ -518,7 +504,7 @@ void releaseSimpleSlot(SimpleSlot simpleSlot) { " when available slots indicated that all slots were available."); } - putIntoMultiMap(slotsForJid, parent.getInstance(), parent); + putIntoMultiMap(slotsForJid, parent.getTaskManagerID(), parent); } } else { // the parent shared slot is now empty and can be released @@ -558,8 +544,6 @@ void releaseSharedSlot(SharedSlot sharedSlot) { /** * *

NOTE: This method must be called from within a scope that holds the lock.

- * - * @param sharedSlot */ private void internalDisposeEmptySharedSlot(SharedSlot sharedSlot) { // sanity check @@ -576,7 +560,7 @@ private void internalDisposeEmptySharedSlot(SharedSlot sharedSlot) { if (parent == null) { // root slot, return to the instance. - sharedSlot.getInstance().returnAllocatedSlot(sharedSlot); + sharedSlot.getOwner().returnAllocatedSlot(sharedSlot); // also, make sure we remove this slot from everywhere allSlots.remove(sharedSlot); @@ -592,7 +576,7 @@ else if (groupID != null) { if (parentRemaining > 0) { // the parent becomes available for the group again - Map> slotsForGroup = availableSlotsPerJid.get(groupID); + Map> slotsForGroup = availableSlotsPerJid.get(groupID); // sanity check if (slotsForGroup == null) { @@ -600,7 +584,7 @@ else if (groupID != null) { " when available slots indicated that all slots were available."); } - putIntoMultiMap(slotsForGroup, parent.getInstance(), parent); + putIntoMultiMap(slotsForGroup, parent.getTaskManagerID(), parent); } else { @@ -620,7 +604,7 @@ else if (groupID != null) { // Utilities // ------------------------------------------------------------------------ - private static void putIntoMultiMap(Map> map, Instance location, SharedSlot slot) { + private static void putIntoMultiMap(Map> map, ResourceID location, SharedSlot slot) { List slotsForInstance = map.get(location); if (slotsForInstance == null) { slotsForInstance = new ArrayList(); @@ -629,7 +613,7 @@ private static void putIntoMultiMap(Map> map, Instanc slotsForInstance.add(slot); } - private static SharedSlot removeFromMultiMap(Map> map, Instance location) { + private static SharedSlot removeFromMultiMap(Map> map, ResourceID location) { List slotsForLocation = map.get(location); if (slotsForLocation == null) { @@ -645,8 +629,8 @@ private static SharedSlot removeFromMultiMap(Map> map } } - private static SharedSlot pollFromMultiMap(Map> map) { - Iterator>> iter = map.entrySet().iterator(); + private static SharedSlot pollFromMultiMap(Map> map) { + Iterator>> iter = map.entrySet().iterator(); while (iter.hasNext()) { List slots = iter.next().getValue(); @@ -667,19 +651,19 @@ else if (slots.size() == 1) { return null; } - private static void removeSlotFromAllEntries(Map>> availableSlots, - SharedSlot slot) + private static void removeSlotFromAllEntries( + Map>> availableSlots, SharedSlot slot) { - final Instance instance = slot.getInstance(); + final ResourceID taskManagerId = slot.getTaskManagerID(); - for (Map.Entry>> entry : availableSlots.entrySet()) { - Map> map = entry.getValue(); + for (Map.Entry>> entry : availableSlots.entrySet()) { + Map> map = entry.getValue(); - List list = map.get(instance); + List list = map.get(taskManagerId); if (list != null) { list.remove(slot); if (list.isEmpty()) { - map.remove(instance); + map.remove(taskManagerId); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java index fece894c3953e..c41f7bfc3c666 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java @@ -18,34 +18,39 @@ package org.apache.flink.runtime.jobmanager.scheduler; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.AbstractID; import org.apache.flink.runtime.instance.Instance; import org.apache.flink.util.Preconditions; import org.apache.flink.runtime.instance.SharedSlot; +import static org.apache.flink.util.Preconditions.checkState; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * A CoLocationConstraint manages the location of a set of tasks * (Execution Vertices). In co-location groups, the different subtasks of * different JobVertices need to be executed on the same {@link Instance}. * This is realized by creating a special shared slot that holds these tasks. * - *

This class tracks the location and the shared slot for this set of tasks.

+ *

This class tracks the location and the shared slot for this set of tasks. */ public class CoLocationConstraint { - + private final CoLocationGroup group; - + private volatile SharedSlot sharedSlot; - - private volatile boolean locationLocked; - - + + private volatile ResourceID lockedLocation; + + CoLocationConstraint(CoLocationGroup group) { Preconditions.checkNotNull(group); this.group = group; } - + // ------------------------------------------------------------------------ // Status & Properties // ------------------------------------------------------------------------ @@ -77,7 +82,7 @@ public AbstractID getGroupId() { * @return True if the location has been assigned, false otherwise. */ public boolean isAssigned() { - return locationLocked; + return lockedLocation != null; } /** @@ -89,7 +94,7 @@ public boolean isAssigned() { * false otherwise. */ public boolean isAssignedAndAlive() { - return locationLocked && sharedSlot.isAlive(); + return lockedLocation != null && sharedSlot.isAlive(); } /** @@ -100,9 +105,9 @@ public boolean isAssignedAndAlive() { * @return The instance describing the location for the tasks of this constraint. * @throws IllegalStateException Thrown if the location has not been assigned, yet. */ - public Instance getLocation() { - if (locationLocked) { - return sharedSlot.getInstance(); + public TaskManagerLocation getLocation() { + if (lockedLocation != null) { + return sharedSlot.getTaskManagerLocation(); } else { throw new IllegalStateException("Location not yet locked"); } @@ -125,18 +130,20 @@ public Instance getLocation() { * the new slot is from a different location. */ public void setSharedSlot(SharedSlot newSlot) { + checkNotNull(newSlot); + if (this.sharedSlot == null) { this.sharedSlot = newSlot; } else if (newSlot != this.sharedSlot){ - if (locationLocked && this.sharedSlot.getInstance() != newSlot.getInstance()) { + if (lockedLocation != null && lockedLocation != newSlot.getTaskManagerID()) { throw new IllegalArgumentException( "Cannot assign different location to a constraint whose location is locked."); } if (this.sharedSlot.isAlive()) { this.sharedSlot.releaseSlot(); } - + this.sharedSlot = newSlot; } } @@ -149,13 +156,10 @@ else if (newSlot != this.sharedSlot){ * or is no slot has been set, yet. */ public void lockLocation() throws IllegalStateException { - if (locationLocked) { - throw new IllegalStateException("Location is already locked"); - } - if (sharedSlot == null) { - throw new IllegalStateException("Cannot lock location without a slot."); - } - locationLocked = true; + checkState(lockedLocation == null, "Location is already locked"); + checkState(sharedSlot != null, "Cannot lock location without a slot."); + + lockedLocation = sharedSlot.getTaskManagerID(); } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java index 963fc4c109b87..b481b55ec12b4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java @@ -24,6 +24,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Queue; @@ -37,6 +38,7 @@ import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.instance.SlotSharingGroupAssignment; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.instance.SharedSlot; @@ -45,6 +47,7 @@ import org.apache.flink.runtime.instance.Instance; import org.apache.flink.runtime.instance.InstanceDiedException; import org.apache.flink.runtime.instance.InstanceListener; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.ExceptionUtils; import org.slf4j.Logger; @@ -78,7 +81,7 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener { private final HashMap> allInstancesByHost = new HashMap>(); /** All instances that still have available resources */ - private final Queue instancesWithAvailableResources = new SetQueue(); + private final Map instancesWithAvailableResources = new LinkedHashMap<>(); /** All tasks pending to be scheduled */ private final Queue taskQueue = new ArrayDeque(); @@ -163,7 +166,7 @@ private Object scheduleTask(ScheduledUnit task, boolean queueIfNoResource) throw final ExecutionVertex vertex = task.getTaskToExecute().getVertex(); - final Iterable preferredLocations = vertex.getPreferredLocations(); + final Iterable preferredLocations = vertex.getPreferredLocations(); final boolean forceExternalLocation = vertex.isScheduleLocalOnly() && preferredLocations != null && preferredLocations.iterator().hasNext(); @@ -222,7 +225,7 @@ private Object scheduleTask(ScheduledUnit task, boolean queueIfNoResource) throw // our location preference is either determined by the location constraint, or by the // vertex's preferred locations - final Iterable locations; + final Iterable locations; final boolean localOnly; if (constraint != null && constraint.isAssigned()) { locations = Collections.singleton(constraint.getLocation()); @@ -341,7 +344,7 @@ else if (forceExternalLocation) { * @return The instance to run the vertex on, it {@code null}, if no instance is available. */ protected SimpleSlot getFreeSlotForTask(ExecutionVertex vertex, - Iterable requestedLocations, + Iterable requestedLocations, boolean localOnly) { // we need potentially to loop multiple times, because there may be false positives // in the set-with-available-instances @@ -360,7 +363,7 @@ protected SimpleSlot getFreeSlotForTask(ExecutionVertex vertex, // if the instance has further available slots, re-add it to the set of available resources. if (instanceToUse.hasResourcesAvailable()) { - this.instancesWithAvailableResources.add(instanceToUse); + this.instancesWithAvailableResources.put(instanceToUse.getResourceId(), instanceToUse); } if (slot != null) { @@ -396,7 +399,7 @@ protected SimpleSlot getFreeSlotForTask(ExecutionVertex vertex, * @return A sub-slot for the given vertex, or {@code null}, if no slot is available. */ protected SimpleSlot getNewSlotForSharingGroup(ExecutionVertex vertex, - Iterable requestedLocations, + Iterable requestedLocations, SlotSharingGroupAssignment groupAssignment, CoLocationConstraint constraint, boolean localOnly) @@ -422,7 +425,7 @@ protected SimpleSlot getNewSlotForSharingGroup(ExecutionVertex vertex, // if the instance has further available slots, re-add it to the set of available resources. if (instanceToUse.hasResourcesAvailable()) { - this.instancesWithAvailableResources.add(instanceToUse); + this.instancesWithAvailableResources.put(instanceToUse.getResourceId(), instanceToUse); } if (sharedSlot != null) { @@ -460,13 +463,13 @@ protected SimpleSlot getNewSlotForSharingGroup(ExecutionVertex vertex, * no locality preference exists. * @param localOnly Flag to indicate whether only one of the exact local instances can be chosen. */ - private Pair findInstance(Iterable requestedLocations, boolean localOnly){ + private Pair findInstance(Iterable requestedLocations, boolean localOnly) { // drain the queue of newly available instances while (this.newlyAvailableInstances.size() > 0) { Instance queuedInstance = this.newlyAvailableInstances.poll(); if (queuedInstance != null) { - this.instancesWithAvailableResources.add(queuedInstance); + this.instancesWithAvailableResources.put(queuedInstance.getResourceId(), queuedInstance); } } @@ -475,15 +478,18 @@ private Pair findInstance(Iterable requestedLocati return null; } - Iterator locations = requestedLocations == null ? null : requestedLocations.iterator(); + Iterator locations = requestedLocations == null ? null : requestedLocations.iterator(); if (locations != null && locations.hasNext()) { // we have a locality preference while (locations.hasNext()) { - Instance location = locations.next(); - if (location != null && this.instancesWithAvailableResources.remove(location)) { - return new ImmutablePair(location, Locality.LOCAL); + TaskManagerLocation location = locations.next(); + if (location != null) { + Instance instance = instancesWithAvailableResources.remove(location.getResourceID()); + if (instance != null) { + return new ImmutablePair(instance, Locality.LOCAL); + } } } @@ -492,14 +498,21 @@ private Pair findInstance(Iterable requestedLocati return null; } else { - Instance instanceToUse = this.instancesWithAvailableResources.poll(); - return new ImmutablePair(instanceToUse, Locality.NON_LOCAL); + // take the first instance from the instances with resources + Iterator instances = instancesWithAvailableResources.values().iterator(); + Instance instanceToUse = instances.next(); + instances.remove(); + + return new ImmutablePair<>(instanceToUse, Locality.NON_LOCAL); } } else { // no location preference, so use some instance - Instance instanceToUse = this.instancesWithAvailableResources.poll(); - return new ImmutablePair(instanceToUse, Locality.UNCONSTRAINED); + Iterator instances = instancesWithAvailableResources.values().iterator(); + Instance instanceToUse = instances.next(); + instances.remove(); + + return new ImmutablePair<>(instanceToUse, Locality.UNCONSTRAINED); } } @@ -570,7 +583,7 @@ private void handleNewSlot() { } } else { - this.instancesWithAvailableResources.add(instance); + this.instancesWithAvailableResources.put(instance.getResourceId(), instance); } } } @@ -643,11 +656,10 @@ public void newInstanceAvailable(Instance instance) { allInstancesByHost.put(instanceHostName, instanceSet); } instanceSet.add(instance); - - + // add it to the available resources and let potential waiters know - this.instancesWithAvailableResources.add(instance); - + this.instancesWithAvailableResources.put(instance.getResourceId(), instance); + // add all slots as available for (int i = 0; i < instance.getNumberOfAvailableSlots(); i++) { newSlotAvailable(instance); @@ -681,8 +693,8 @@ private void removeInstance(Instance instance) { } allInstances.remove(instance); - instancesWithAvailableResources.remove(instance); - + instancesWithAvailableResources.remove(instance.getResourceId()); + String instanceHostName = instance.getInstanceConnectionInfo().getHostname(); Set instanceSet = allInstancesByHost.get(instanceHostName); if (instanceSet != null) { @@ -709,7 +721,7 @@ public int getNumberOfAvailableSlots() { synchronized (globalLock) { processNewlyAvailableInstances(); - for (Instance instance : instancesWithAvailableResources) { + for (Instance instance : instancesWithAvailableResources.values()) { count += instance.getNumberOfAvailableSlots(); } } @@ -781,9 +793,9 @@ private void processNewlyAvailableInstances() { synchronized (globalLock) { Instance instance; - while((instance = newlyAvailableInstances.poll()) != null){ - if(instance.hasResourcesAvailable()){ - instancesWithAvailableResources.add(instance); + while ((instance = newlyAvailableInstances.poll()) != null) { + if (instance.hasResourcesAvailable()) { + instancesWithAvailableResources.put(instance.getResourceId(), instance); } } } @@ -794,17 +806,17 @@ private void processNewlyAvailableInstances() { // Utilities // ------------------------------------------------------------------------ - private static String getHostnamesFromInstances(Iterable instances) { + private static String getHostnamesFromInstances(Iterable locations) { StringBuilder bld = new StringBuilder(); boolean successive = false; - for (Instance i : instances) { + for (TaskManagerLocation loc : locations) { if (successive) { bld.append(", "); } else { successive = true; } - bld.append(i.getInstanceConnectionInfo().getHostname()); + bld.append(loc.getHostname()); } return bld.toString(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFuture.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFuture.java index 31bd3417f84c7..36e4072a620a6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFuture.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFuture.java @@ -20,73 +20,125 @@ import org.apache.flink.runtime.instance.SimpleSlot; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * + */ public class SlotAllocationFuture { - + private final Object monitor = new Object(); - + private volatile SimpleSlot slot; - + private volatile SlotAllocationFutureAction action; - + // -------------------------------------------------------------------------------------------- + /** + * Creates a future that is uncompleted. + */ public SlotAllocationFuture() {} - + + /** + * Creates a future that is immediately completed. + * + * @param slot The task slot that completes the future. + */ public SlotAllocationFuture(SimpleSlot slot) { this.slot = slot; } - + // -------------------------------------------------------------------------------------------- - - public SimpleSlot waitTillAllocated() throws InterruptedException { - return waitTillAllocated(0); - } - - public SimpleSlot waitTillAllocated(long timeout) throws InterruptedException { + + public SimpleSlot waitTillCompleted() throws InterruptedException { synchronized (monitor) { while (slot == null) { - monitor.wait(timeout); + monitor.wait(); + } + return slot; + } + } + + public SimpleSlot waitTillCompleted(long timeout, TimeUnit timeUnit) throws InterruptedException, TimeoutException { + checkArgument(timeout >= 0, "timeout may not be negative"); + checkNotNull(timeUnit, "timeUnit"); + + if (timeout == 0) { + return waitTillCompleted(); + } else { + final long deadline = System.nanoTime() + timeUnit.toNanos(timeout); + long millisToWait; + + synchronized (monitor) { + while (slot == null && (millisToWait = (deadline - System.nanoTime()) / 1_000_000) > 0) { + monitor.wait(millisToWait); + } + + if (slot != null) { + return slot; + } else { + throw new TimeoutException(); + } } - + } + } + + /** + * Gets the slot from this future. This method throws an exception, if the future has not been completed. + * This method never blocks. + * + * @return The slot with which this future was completed. + * @throws IllegalStateException Thrown, if this method is called before the future is completed. + */ + public SimpleSlot get() { + final SimpleSlot slot = this.slot; + if (slot != null) { return slot; + } else { + throw new IllegalStateException("The future is not complete - not slot available"); } } - + public void setFutureAction(SlotAllocationFutureAction action) { + checkNotNull(action); + synchronized (monitor) { - if (this.action != null) { - throw new IllegalStateException("Future already has an action registered."); - } - + checkState(this.action == null, "Future already has an action registered."); + this.action = action; - + if (this.slot != null) { action.slotAllocated(this.slot); } } } - + + /** + * Completes the future with a slot. + */ public void setSlot(SimpleSlot slot) { - if (slot == null) { - throw new NullPointerException(); - } - + checkNotNull(slot); + synchronized (monitor) { - if (this.slot != null) { - throw new IllegalStateException("The future has already been assigned a slot."); - } - + checkState(this.slot == null, "The future has already been assigned a slot."); + this.slot = slot; monitor.notifyAll(); - + if (action != null) { action.slotAllocated(slot); } } } - + // -------------------------------------------------------------------------------------------- - + @Override public String toString() { return slot == null ? "PENDING" : "DONE"; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/SlotOwner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/SlotOwner.java new file mode 100644 index 0000000000000..ad9c784c3407e --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/SlotOwner.java @@ -0,0 +1,29 @@ +/* + * 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.runtime.jobmanager.slots; + +import org.apache.flink.runtime.instance.Slot; + +/** + * Interface for components that hold slots and to which slots get released / recycled. + */ +public interface SlotOwner { + + boolean returnAllocatedSlot(Slot slot); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerLocation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerLocation.java index 5a0faa55a72d1..01d06542032a4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerLocation.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerLocation.java @@ -108,7 +108,7 @@ public TaskManagerLocation(ResourceID resourceID, InetAddress inetAddress, int d } this.stringRepresentation = String.format( - "TaskManager (%s) @ %s (dataPort=%d)", resourceID, fqdnHostName, dataPort); + "%s @ %s (dataPort=%d)", resourceID, fqdnHostName, dataPort); } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index 0c62c69b002ce..2a0ecc2e36545 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -650,7 +650,7 @@ class JobManager( val taskId = execution.getVertex.getParallelSubtaskIndex val host = if (slot != null) { - slot.getInstance().getInstanceConnectionInfo.getHostname + slot.getTaskManagerLocation().getHostname() } else { null } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala index 6a9b4904f35da..3947b1738260b 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala @@ -249,7 +249,7 @@ trait TestingJobManagerLike extends FlinkActor { } else { sender ! decorateMessage( WorkingTaskManager( - Some(resource.getInstance().getActorGateway) + Some(resource.getTaskManagerActorGateway()) ) ) } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java index cf7cf5800ec7c..d8bd6cbe9cad5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java @@ -117,7 +117,8 @@ public void testExecutionGraphRestartTimeMetric() throws JobException, IOExcepti ActorGateway actorGateway = mock(ActorGateway.class); when(simpleSlot.isAlive()).thenReturn(true); - when(simpleSlot.getInstance()).thenReturn(instance); + when(simpleSlot.getTaskManagerID()).thenReturn(instance.getResourceId()); + when(simpleSlot.getTaskManagerLocation()).thenReturn(instance.getInstanceConnectionInfo()); when(simpleSlot.setExecutedVertex(Matchers.any(Execution.class))).thenReturn(true); when(simpleSlot.getRoot()).thenReturn(rootSlot); @@ -152,6 +153,7 @@ public void testExecutionGraphRestartTimeMetric() throws JobException, IOExcepti assertNotNull(metric); assertTrue(metric instanceof Gauge); + @SuppressWarnings("unchecked") Gauge restartingTime = (Gauge) metric; // check that the restarting time is 0 since it's the initial start diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java index 91472ae2fdbb5..a1f3345541b53 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java @@ -92,8 +92,8 @@ public void testScheduleWithConstraint1() { ExecutionJobVertex ejv = eg.getAllVertices().get(jobVertex.getID()); ExecutionVertex[] vertices = ejv.getTaskVertices(); - vertices[0].setLocationConstraintHosts(Arrays.asList(instance1, instance2)); - vertices[1].setLocationConstraintHosts(Collections.singletonList(instance3)); + vertices[0].setLocationConstraintHosts(Arrays.asList(instance1.getInstanceConnectionInfo(), instance2.getInstanceConnectionInfo())); + vertices[1].setLocationConstraintHosts(Collections.singletonList(instance3.getInstanceConnectionInfo())); vertices[0].setScheduleLocalOnly(true); vertices[1].setScheduleLocalOnly(true); @@ -106,14 +106,14 @@ public void testScheduleWithConstraint1() { assertNotNull(slot1); assertNotNull(slot2); - Instance target1 = slot1.getInstance(); - Instance target2 = slot2.getInstance(); + ResourceID target1 = slot1.getTaskManagerID(); + ResourceID target2 = slot2.getTaskManagerID(); assertNotNull(target1); assertNotNull(target2); - assertTrue(target1 == instance1 || target1 == instance2); - assertTrue(target2 == instance3); + assertTrue(target1 == instance1.getResourceId() || target1 == instance2.getResourceId()); + assertEquals(target2, instance3.getResourceId()); } catch (Exception e) { e.printStackTrace(); @@ -165,8 +165,8 @@ public void testScheduleWithConstraint2() { ExecutionJobVertex ejv = eg.getAllVertices().get(jobVertex.getID()); ExecutionVertex[] vertices = ejv.getTaskVertices(); - vertices[0].setLocationConstraintHosts(Collections.singletonList(instance3)); - vertices[1].setLocationConstraintHosts(Arrays.asList(instance1, instance2)); + vertices[0].setLocationConstraintHosts(Collections.singletonList(instance3.getInstanceConnectionInfo())); + vertices[1].setLocationConstraintHosts(Arrays.asList(instance1.getInstanceConnectionInfo(), instance2.getInstanceConnectionInfo())); vertices[0].setScheduleLocalOnly(true); vertices[1].setScheduleLocalOnly(true); @@ -179,14 +179,11 @@ public void testScheduleWithConstraint2() { assertNotNull(slot1); assertNotNull(slot2); - Instance target1 = slot1.getInstance(); - Instance target2 = slot2.getInstance(); + ResourceID target1 = slot1.getTaskManagerID(); + ResourceID target2 = slot2.getTaskManagerID(); - assertNotNull(target1); - assertNotNull(target2); - - assertTrue(target1 == instance3); - assertTrue(target2 == instance1 || target2 == instance2); + assertTrue(target1 == instance3.getResourceId()); + assertTrue(target2 == instance1.getResourceId() || target2 == instance2.getResourceId()); } catch (Exception e) { e.printStackTrace(); @@ -242,8 +239,8 @@ public void testScheduleWithConstraintAndSlotSharing() { ExecutionJobVertex ejv = eg.getAllVertices().get(jobVertex1.getID()); ExecutionVertex[] vertices = ejv.getTaskVertices(); - vertices[0].setLocationConstraintHosts(Arrays.asList(instance1, instance2)); - vertices[1].setLocationConstraintHosts(Collections.singletonList(instance3)); + vertices[0].setLocationConstraintHosts(Arrays.asList(instance1.getInstanceConnectionInfo(), instance2.getInstanceConnectionInfo())); + vertices[1].setLocationConstraintHosts(Collections.singletonList(instance3.getInstanceConnectionInfo())); vertices[0].setScheduleLocalOnly(true); vertices[1].setScheduleLocalOnly(true); @@ -255,15 +252,12 @@ public void testScheduleWithConstraintAndSlotSharing() { assertNotNull(slot1); assertNotNull(slot2); - - Instance target1 = slot1.getInstance(); - Instance target2 = slot2.getInstance(); - - assertNotNull(target1); - assertNotNull(target2); - - assertTrue(target1 == instance1 || target1 == instance2); - assertTrue(target2 == instance3); + + ResourceID target1 = slot1.getTaskManagerID(); + ResourceID target2 = slot2.getTaskManagerID(); + + assertTrue(target1 == instance1.getResourceId() || target1 == instance2.getResourceId()); + assertTrue(target2 == instance3.getResourceId()); } catch (Exception e) { e.printStackTrace(); @@ -310,7 +304,7 @@ public void testScheduleWithUnfulfillableConstraint() { ExecutionJobVertex ejv = eg.getAllVertices().get(jobVertex.getID()); ExecutionVertex[] vertices = ejv.getTaskVertices(); - vertices[0].setLocationConstraintHosts(Collections.singletonList(instance2)); + vertices[0].setLocationConstraintHosts(Collections.singletonList(instance2.getInstanceConnectionInfo())); vertices[0].setScheduleLocalOnly(true); try { @@ -380,7 +374,7 @@ public void testScheduleWithUnfulfillableConstraintInSharingGroup() { ExecutionJobVertex ejv = eg.getAllVertices().get(jobVertex1.getID()); ExecutionVertex[] vertices = ejv.getTaskVertices(); - vertices[0].setLocationConstraintHosts(Collections.singletonList(instance2)); + vertices[0].setLocationConstraintHosts(Collections.singletonList(instance2.getInstanceConnectionInfo())); vertices[0].setScheduleLocalOnly(true); try { @@ -420,7 +414,7 @@ public void testArchivingClearsFields() { ExecutionVertex ev = eg.getAllVertices().get(vertex.getID()).getTaskVertices()[0]; Instance instance = ExecutionGraphTestUtils.getInstance(DummyActorGateway.INSTANCE); - ev.setLocationConstraintHosts(Collections.singletonList(instance)); + ev.setLocationConstraintHosts(Collections.singletonList(instance.getInstanceConnectionInfo())); assertNotNull(ev.getPreferredLocations()); assertEquals(instance, ev.getPreferredLocations().iterator().next()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SharedSlotsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SharedSlotsTest.java index 8709395edd9f7..2c40e89325dd4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SharedSlotsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SharedSlotsTest.java @@ -26,6 +26,7 @@ import org.apache.flink.runtime.jobmanager.scheduler.Locality; import org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.AbstractID; import org.junit.Test; @@ -38,7 +39,9 @@ * Tests for the allocation, properties, and release of shared slots. */ public class SharedSlotsTest { - + + private static final Iterable NO_LOCATION = Collections.emptySet(); + @Test public void allocateAndReleaseEmptySlot() { try { @@ -116,20 +119,20 @@ public void allocateSimpleSlotsAndReleaseFromRoot() { SlotSharingGroupAssignment assignment = sharingGroup.getTaskAssignment(); Instance instance = SchedulerTestUtils.getRandomInstance(1); - + // allocate a shared slot SharedSlot sharedSlot = instance.allocateSharedSlot(jobId, assignment); - + // allocate a series of sub slots - + SimpleSlot sub1 = assignment.addSharedSlotAndAllocateSubSlot(sharedSlot, Locality.LOCAL, vid1); assertNotNull(sub1); - + assertNull(sub1.getExecutedVertex()); assertEquals(Locality.LOCAL, sub1.getLocality()); assertEquals(1, sub1.getNumberLeaves()); assertEquals(vid1, sub1.getGroupID()); - assertEquals(instance, sub1.getInstance()); + assertEquals(instance.getResourceId(), sub1.getTaskManagerID()); assertEquals(jobId, sub1.getJobID()); assertEquals(sharedSlot, sub1.getParent()); assertEquals(sharedSlot, sub1.getRoot()); @@ -141,14 +144,14 @@ public void allocateSimpleSlotsAndReleaseFromRoot() { assertEquals(1, assignment.getNumberOfAvailableSlotsForGroup(vid3)); assertEquals(1, assignment.getNumberOfAvailableSlotsForGroup(vid4)); - SimpleSlot sub2 = assignment.getSlotForTask(vid2, Collections.emptySet()); + SimpleSlot sub2 = assignment.getSlotForTask(vid2, NO_LOCATION); assertNotNull(sub2); assertNull(sub2.getExecutedVertex()); assertEquals(Locality.UNCONSTRAINED, sub2.getLocality()); assertEquals(1, sub2.getNumberLeaves()); assertEquals(vid2, sub2.getGroupID()); - assertEquals(instance, sub2.getInstance()); + assertEquals(instance.getResourceId(), sub2.getTaskManagerID()); assertEquals(jobId, sub2.getJobID()); assertEquals(sharedSlot, sub2.getParent()); assertEquals(sharedSlot, sub2.getRoot()); @@ -160,14 +163,14 @@ public void allocateSimpleSlotsAndReleaseFromRoot() { assertEquals(1, assignment.getNumberOfAvailableSlotsForGroup(vid3)); assertEquals(1, assignment.getNumberOfAvailableSlotsForGroup(vid4)); - SimpleSlot sub3 = assignment.getSlotForTask(vid3, Collections.singleton(instance)); + SimpleSlot sub3 = assignment.getSlotForTask(vid3, Collections.singleton(instance.getInstanceConnectionInfo())); assertNotNull(sub3); assertNull(sub3.getExecutedVertex()); assertEquals(Locality.LOCAL, sub3.getLocality()); assertEquals(1, sub3.getNumberLeaves()); assertEquals(vid3, sub3.getGroupID()); - assertEquals(instance, sub3.getInstance()); + assertEquals(instance.getResourceId(), sub3.getTaskManagerID()); assertEquals(jobId, sub3.getJobID()); assertEquals(sharedSlot, sub3.getParent()); assertEquals(sharedSlot, sub3.getRoot()); @@ -180,14 +183,14 @@ public void allocateSimpleSlotsAndReleaseFromRoot() { assertEquals(1, assignment.getNumberOfAvailableSlotsForGroup(vid4)); SimpleSlot sub4 = assignment.getSlotForTask(vid4, - Collections.singleton(SchedulerTestUtils.getRandomInstance(1))); + Collections.singleton(SchedulerTestUtils.getRandomInstance(1).getInstanceConnectionInfo())); assertNotNull(sub4); assertNull(sub4.getExecutedVertex()); assertEquals(Locality.NON_LOCAL, sub4.getLocality()); assertEquals(1, sub4.getNumberLeaves()); assertEquals(vid4, sub4.getGroupID()); - assertEquals(instance, sub4.getInstance()); + assertEquals(instance.getResourceId(), sub4.getTaskManagerID()); assertEquals(jobId, sub4.getJobID()); assertEquals(sharedSlot, sub4.getParent()); assertEquals(sharedSlot, sub4.getRoot()); @@ -229,7 +232,7 @@ public void allocateSimpleSlotsAndReleaseFromRoot() { } @Test - public void allocateSimpleSlotsAndReleaseFromleaves() { + public void allocateSimpleSlotsAndReleaseFromLeaves() { try { JobID jobId = new JobID(); JobVertexID vid1 = new JobVertexID(); @@ -247,8 +250,8 @@ public void allocateSimpleSlotsAndReleaseFromleaves() { // allocate a series of sub slots SimpleSlot sub1 = assignment.addSharedSlotAndAllocateSubSlot(sharedSlot, Locality.UNCONSTRAINED, vid1); - SimpleSlot sub2 = assignment.getSlotForTask(vid2, Collections.emptySet()); - SimpleSlot sub3 = assignment.getSlotForTask(vid3, Collections.emptySet()); + SimpleSlot sub2 = assignment.getSlotForTask(vid2, NO_LOCATION); + SimpleSlot sub3 = assignment.getSlotForTask(vid3, NO_LOCATION); assertNotNull(sub1); assertNotNull(sub2); @@ -332,7 +335,7 @@ public void allocateAndReleaseInMixedOrder() { // allocate a series of sub slots SimpleSlot sub1 = assignment.addSharedSlotAndAllocateSubSlot(sharedSlot, Locality.UNCONSTRAINED, vid1); - SimpleSlot sub2 = assignment.getSlotForTask(vid2, Collections.emptySet()); + SimpleSlot sub2 = assignment.getSlotForTask(vid2, NO_LOCATION); assertNotNull(sub1); assertNotNull(sub2); @@ -353,7 +356,7 @@ public void allocateAndReleaseInMixedOrder() { assertEquals(1, assignment.getNumberOfSlots()); - SimpleSlot sub3 = assignment.getSlotForTask(vid3, Collections.emptySet()); + SimpleSlot sub3 = assignment.getSlotForTask(vid3, NO_LOCATION); assertNotNull(sub3); assertEquals(2, sharedSlot.getNumberLeaves()); @@ -431,7 +434,7 @@ public void testAllocateAndReleaseTwoLevels() { assertEquals(1, sharedSlot.getNumberLeaves()); // get the first slot in the nested shared slot from the co-location constraint - SimpleSlot headSlot = assignment.getSlotForTask(constraint, Collections.emptySet()); + SimpleSlot headSlot = assignment.getSlotForTask(constraint, Collections.emptySet()); assertEquals(2, sharedSlot.getNumberLeaves()); assertNotNull(constraint.getSharedSlot()); @@ -447,20 +450,20 @@ public void testAllocateAndReleaseTwoLevels() { assertFalse(constraint.isAssigned()); // re-allocate the head slot - headSlot = assignment.getSlotForTask(constraint, Collections.emptySet()); + headSlot = assignment.getSlotForTask(constraint, Collections.emptySet()); constraint.lockLocation(); assertNotNull(constraint.getSharedSlot()); assertTrue(constraint.isAssigned()); assertTrue(constraint.isAssignedAndAlive()); - assertEquals(instance, constraint.getLocation()); + assertEquals(instance.getInstanceConnectionInfo(), constraint.getLocation()); - SimpleSlot tailSlot = assignment.getSlotForTask(constraint, Collections.emptySet()); + SimpleSlot tailSlot = assignment.getSlotForTask(constraint, Collections.emptySet()); assertEquals(constraint.getSharedSlot(), headSlot.getParent()); assertEquals(constraint.getSharedSlot(), tailSlot.getParent()); - SimpleSlot sinkSlot = assignment.getSlotForTask(sinkId, Collections.emptySet()); + SimpleSlot sinkSlot = assignment.getSlotForTask(sinkId, Collections.emptySet()); assertEquals(4, sharedSlot.getNumberLeaves()); // we release our co-location constraint tasks @@ -472,23 +475,23 @@ public void testAllocateAndReleaseTwoLevels() { assertTrue(tailSlot.isReleased()); assertTrue(constraint.isAssigned()); assertFalse(constraint.isAssignedAndAlive()); - assertEquals(instance, constraint.getLocation()); + assertEquals(instance.getInstanceConnectionInfo(), constraint.getLocation()); // we should have resources again for the co-location constraint assertEquals(1, assignment.getNumberOfAvailableSlotsForGroup(constraint.getGroupId())); // re-allocate head and tail from the constraint - headSlot = assignment.getSlotForTask(constraint, Collections.emptySet()); - tailSlot = assignment.getSlotForTask(constraint, Collections.emptySet()); + headSlot = assignment.getSlotForTask(constraint, NO_LOCATION); + tailSlot = assignment.getSlotForTask(constraint, NO_LOCATION); assertEquals(4, sharedSlot.getNumberLeaves()); assertEquals(0, assignment.getNumberOfAvailableSlotsForGroup(constraint.getGroupId())); // verify some basic properties of the slots - assertEquals(instance, sourceSlot.getInstance()); - assertEquals(instance, headSlot.getInstance()); - assertEquals(instance, tailSlot.getInstance()); - assertEquals(instance, sinkSlot.getInstance()); + assertEquals(instance.getResourceId(), sourceSlot.getTaskManagerID()); + assertEquals(instance.getResourceId(), headSlot.getTaskManagerID()); + assertEquals(instance.getResourceId(), tailSlot.getTaskManagerID()); + assertEquals(instance.getResourceId(), sinkSlot.getTaskManagerID()); assertEquals(sourceId, sourceSlot.getGroupID()); assertEquals(sinkId, sinkSlot.getGroupID()); @@ -564,11 +567,11 @@ public void testReleaseTwoLevelsFromRoot() { // get the first simple slot SimpleSlot sourceSlot = assignment.addSharedSlotAndAllocateSubSlot(sharedSlot, Locality.LOCAL, sourceId); - SimpleSlot headSlot = assignment.getSlotForTask(constraint, Collections.emptySet()); + SimpleSlot headSlot = assignment.getSlotForTask(constraint, NO_LOCATION); constraint.lockLocation(); - SimpleSlot tailSlot = assignment.getSlotForTask(constraint, Collections.emptySet()); + SimpleSlot tailSlot = assignment.getSlotForTask(constraint, NO_LOCATION); - SimpleSlot sinkSlot = assignment.getSlotForTask(sinkId, Collections.emptySet()); + SimpleSlot sinkSlot = assignment.getSlotForTask(sinkId, NO_LOCATION); assertEquals(4, sharedSlot.getNumberLeaves()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java index 676b2a34e1c6f..5b7d18a3c20ea 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java @@ -27,9 +27,11 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.instance.Instance; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.junit.Test; @@ -41,17 +43,17 @@ public void scheduleAllSharedAndCoLocated() { try { JobVertexID jid1 = new JobVertexID(); JobVertexID jid2 = new JobVertexID(); - + Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext()); - + scheduler.newInstanceAvailable(getRandomInstance(2)); scheduler.newInstanceAvailable(getRandomInstance(2)); scheduler.newInstanceAvailable(getRandomInstance(2)); - + assertEquals(6, scheduler.getNumberOfAvailableSlots()); - + SlotSharingGroup sharingGroup = new SlotSharingGroup(); - + CoLocationGroup ccg = new CoLocationGroup(); CoLocationConstraint c1 = new CoLocationConstraint(ccg); CoLocationConstraint c2 = new CoLocationConstraint(ccg); @@ -59,7 +61,7 @@ public void scheduleAllSharedAndCoLocated() { CoLocationConstraint c4 = new CoLocationConstraint(ccg); CoLocationConstraint c5 = new CoLocationConstraint(ccg); CoLocationConstraint c6 = new CoLocationConstraint(ccg); - + // schedule 4 tasks from the first vertex group SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 6), sharingGroup, c1)); SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 1, 6), sharingGroup, c2)); @@ -86,7 +88,7 @@ public void scheduleAllSharedAndCoLocated() { assertNotNull(s10); assertNotNull(s11); assertNotNull(s12); - + // check that each slot got exactly two tasks assertEquals(2, s1.getRoot().getNumberLeaves()); assertEquals(2, s2.getRoot().getNumberLeaves()); @@ -100,20 +102,20 @@ public void scheduleAllSharedAndCoLocated() { assertEquals(2, s10.getRoot().getNumberLeaves()); assertEquals(2, s11.getRoot().getNumberLeaves()); assertEquals(2, s12.getRoot().getNumberLeaves()); - - assertEquals(s1.getInstance(), s5.getInstance()); - assertEquals(s2.getInstance(), s6.getInstance()); - assertEquals(s3.getInstance(), s7.getInstance()); - assertEquals(s4.getInstance(), s10.getInstance()); - assertEquals(s8.getInstance(), s11.getInstance()); - assertEquals(s9.getInstance(), s12.getInstance()); - - assertEquals(c1.getLocation(), s1.getInstance()); - assertEquals(c2.getLocation(), s2.getInstance()); - assertEquals(c3.getLocation(), s3.getInstance()); - assertEquals(c4.getLocation(), s4.getInstance()); - assertEquals(c5.getLocation(), s8.getInstance()); - assertEquals(c6.getLocation(), s9.getInstance()); + + assertEquals(s1.getTaskManagerID(), s5.getTaskManagerID()); + assertEquals(s2.getTaskManagerID(), s6.getTaskManagerID()); + assertEquals(s3.getTaskManagerID(), s7.getTaskManagerID()); + assertEquals(s4.getTaskManagerID(), s10.getTaskManagerID()); + assertEquals(s8.getTaskManagerID(), s11.getTaskManagerID()); + assertEquals(s9.getTaskManagerID(), s12.getTaskManagerID()); + + assertEquals(c1.getLocation(), s1.getTaskManagerLocation()); + assertEquals(c2.getLocation(), s2.getTaskManagerLocation()); + assertEquals(c3.getLocation(), s3.getTaskManagerLocation()); + assertEquals(c4.getLocation(), s4.getTaskManagerLocation()); + assertEquals(c5.getLocation(), s8.getTaskManagerLocation()); + assertEquals(c6.getLocation(), s9.getTaskManagerLocation()); // check the scheduler's bookkeeping assertEquals(0, scheduler.getNumberOfAvailableSlots()); @@ -186,14 +188,14 @@ public void scheduleWithIntermediateRelease() { SimpleSlot sSolo = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 0, 1))); - Instance loc = s1.getInstance(); + ResourceID taskManager = s1.getTaskManagerID(); s1.releaseSlot(); s2.releaseSlot(); sSolo.releaseSlot(); SimpleSlot sNew = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 0, 1), sharingGroup, c1)); - assertEquals(loc, sNew.getInstance()); + assertEquals(taskManager, sNew.getTaskManagerID()); assertEquals(2, scheduler.getNumberOfLocalizedAssignments()); assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments()); @@ -296,10 +298,10 @@ public void scheduleMixedCoLocationSlotSharing() { scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 2, 4), shareGroup)); scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 3, 4), shareGroup)); - assertEquals(s21.getInstance(), s34.getInstance()); - assertEquals(s22.getInstance(), s31.getInstance()); - assertEquals(s23.getInstance(), s32.getInstance()); - assertEquals(s24.getInstance(), s33.getInstance()); + assertEquals(s21.getTaskManagerID(), s34.getTaskManagerID()); + assertEquals(s22.getTaskManagerID(), s31.getTaskManagerID()); + assertEquals(s23.getTaskManagerID(), s32.getTaskManagerID()); + assertEquals(s24.getTaskManagerID(), s33.getTaskManagerID()); assertEquals(4, scheduler.getNumberOfLocalizedAssignments()); assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments()); @@ -320,10 +322,13 @@ public void testGetsNonLocalFromSharingGroupFirst() { JobVertexID jid3 = new JobVertexID(); Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext()); - + Instance i1 = getRandomInstance(1); Instance i2 = getRandomInstance(1); - + + TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); + TaskManagerLocation loc2 = i2.getInstanceConnectionInfo(); + scheduler.newInstanceAvailable(i2); scheduler.newInstanceAvailable(i1); @@ -336,29 +341,29 @@ public void testGetsNonLocalFromSharingGroupFirst() { CoLocationConstraint cc2 = new CoLocationConstraint(ccg); // schedule something into the shared group so that both instances are in the sharing group - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, i1), sharingGroup)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, i2), sharingGroup)); + SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup)); + SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup)); // schedule one locally to instance 1 - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, i1), sharingGroup, cc1)); + SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup, cc1)); // schedule with co location constraint (yet unassigned) and a preference for // instance 1, but it can only get instance 2 - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, i1), sharingGroup, cc2)); + SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup, cc2)); // schedule something into the assigned co-location constraints and check that they override the // other preferences - SimpleSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid3, 0, 2, i2), sharingGroup, cc1)); - SimpleSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid3, 1, 2, i1), sharingGroup, cc2)); + SimpleSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid3, 0, 2, loc2), sharingGroup, cc1)); + SimpleSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid3, 1, 2, loc1), sharingGroup, cc2)); // check that each slot got three assertEquals(3, s1.getRoot().getNumberLeaves()); assertEquals(3, s2.getRoot().getNumberLeaves()); - assertEquals(s1.getInstance(), s3.getInstance()); - assertEquals(s2.getInstance(), s4.getInstance()); - assertEquals(s1.getInstance(), s5.getInstance()); - assertEquals(s2.getInstance(), s6.getInstance()); + assertEquals(s1.getTaskManagerID(), s3.getTaskManagerID()); + assertEquals(s2.getTaskManagerID(), s4.getTaskManagerID()); + assertEquals(s1.getTaskManagerID(), s5.getTaskManagerID()); + assertEquals(s2.getTaskManagerID(), s6.getTaskManagerID()); // check the scheduler's bookkeeping assertEquals(0, scheduler.getNumberOfAvailableSlots()); @@ -389,10 +394,13 @@ public void testSlotReleasedInBetween() { JobVertexID jid2 = new JobVertexID(); Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext()); - + Instance i1 = getRandomInstance(1); Instance i2 = getRandomInstance(1); - + + TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); + TaskManagerLocation loc2 = i2.getInstanceConnectionInfo(); + scheduler.newInstanceAvailable(i2); scheduler.newInstanceAvailable(i1); @@ -404,8 +412,8 @@ public void testSlotReleasedInBetween() { CoLocationConstraint cc1 = new CoLocationConstraint(ccg); CoLocationConstraint cc2 = new CoLocationConstraint(ccg); - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, i1), sharingGroup, cc1)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, i2), sharingGroup, cc2)); + SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup, cc1)); + SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup, cc2)); s1.releaseSlot(); s2.releaseSlot(); @@ -413,12 +421,12 @@ public void testSlotReleasedInBetween() { assertEquals(2, scheduler.getNumberOfAvailableSlots()); assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfSlots()); - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, i2), sharingGroup, cc1)); - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, i1), sharingGroup, cc2)); + SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc2), sharingGroup, cc1)); + SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup, cc2)); // still preserves the previous instance mapping) - assertEquals(i1, s3.getInstance()); - assertEquals(i2, s4.getInstance()); + assertEquals(i1.getResourceId(), s3.getTaskManagerID()); + assertEquals(i2.getResourceId(), s4.getTaskManagerID()); s3.releaseSlot(); s4.releaseSlot(); @@ -443,10 +451,13 @@ public void testSlotReleasedInBetweenAndNoNewLocal() { JobVertexID jidx = new JobVertexID(); Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext()); - + Instance i1 = getRandomInstance(1); Instance i2 = getRandomInstance(1); - + + TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); + TaskManagerLocation loc2 = i2.getInstanceConnectionInfo(); + scheduler.newInstanceAvailable(i2); scheduler.newInstanceAvailable(i1); @@ -458,8 +469,8 @@ public void testSlotReleasedInBetweenAndNoNewLocal() { CoLocationConstraint cc1 = new CoLocationConstraint(ccg); CoLocationConstraint cc2 = new CoLocationConstraint(ccg); - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, i1), sharingGroup, cc1)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, i2), sharingGroup, cc2)); + SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup, cc1)); + SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup, cc2)); s1.releaseSlot(); s2.releaseSlot(); @@ -471,7 +482,7 @@ public void testSlotReleasedInBetweenAndNoNewLocal() { SimpleSlot sb = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jidx, 1, 2))); try { - scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, i2), sharingGroup, cc1)); + scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc2), sharingGroup, cc1)); fail("should not be able to find a resource"); } catch (NoResourceAvailableException e) { // good @@ -504,7 +515,9 @@ public void testScheduleOutOfOrder() { Instance i1 = getRandomInstance(1); Instance i2 = getRandomInstance(1); - + + TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); + scheduler.newInstanceAvailable(i2); scheduler.newInstanceAvailable(i1); @@ -519,18 +532,18 @@ public void testScheduleOutOfOrder() { // schedule something from the second job vertex id before the first is filled, // and give locality preferences that hint at using the same shared slot for both // co location constraints (which we seek to prevent) - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, i1), sharingGroup, cc1)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, i1), sharingGroup, cc2)); + SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup, cc1)); + SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup, cc2)); - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, i1), sharingGroup, cc1)); - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, i1), sharingGroup, cc2)); + SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup, cc1)); + SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc1), sharingGroup, cc2)); // check that each slot got three assertEquals(2, s1.getRoot().getNumberLeaves()); assertEquals(2, s2.getRoot().getNumberLeaves()); - assertEquals(s1.getInstance(), s3.getInstance()); - assertEquals(s2.getInstance(), s4.getInstance()); + assertEquals(s1.getTaskManagerID(), s3.getTaskManagerID()); + assertEquals(s2.getTaskManagerID(), s4.getTaskManagerID()); // check the scheduler's bookkeeping assertEquals(0, scheduler.getNumberOfAvailableSlots()); @@ -566,7 +579,10 @@ public void nonColocationFollowsCoLocation() { Instance i1 = getRandomInstance(1); Instance i2 = getRandomInstance(1); - + + TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); + TaskManagerLocation loc2 = i2.getInstanceConnectionInfo(); + scheduler.newInstanceAvailable(i2); scheduler.newInstanceAvailable(i1); @@ -578,11 +594,11 @@ public void nonColocationFollowsCoLocation() { CoLocationConstraint cc1 = new CoLocationConstraint(ccg); CoLocationConstraint cc2 = new CoLocationConstraint(ccg); - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, i1), sharingGroup, cc1)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, i2), sharingGroup, cc2)); + SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup, cc1)); + SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup, cc2)); - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, i1), sharingGroup)); - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, i1), sharingGroup)); + SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup)); + SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup)); // check that each slot got two assertEquals(2, s1.getRoot().getNumberLeaves()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java index 2ee53d93601c3..25498c4e0ef0c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java @@ -18,26 +18,28 @@ package org.apache.flink.runtime.jobmanager.scheduler; -import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.areAllDistinct; -import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getDummyTask; -import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertex; -import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getRandomInstance; -import static org.junit.Assert.*; - +import org.apache.flink.runtime.instance.Instance; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.testingUtils.TestingUtils; + import org.junit.Test; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.flink.runtime.instance.Instance; +import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.areAllDistinct; +import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getDummyTask; +import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getRandomInstance; +import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertex; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; /** * Tests for the {@link Scheduler} when scheduling individual tasks. @@ -254,7 +256,7 @@ public void run() { List slotsAfter = new ArrayList(); for (SlotAllocationFuture future : allAllocatedSlots) { - slotsAfter.add(future.waitTillAllocated()); + slotsAfter.add(future.waitTillCompleted()); } assertEquals("All instances should have available slots.", NUM_INSTANCES, @@ -294,7 +296,7 @@ public void testScheduleWithDyingInstances() { i2.markDead(); for (SimpleSlot slot : slots) { - if (slot.getInstance() == i2) { + if (slot.getOwner() == i2) { assertTrue(slot.isCanceled()); } else { assertFalse(slot.isCanceled()); @@ -345,37 +347,37 @@ public void testSchedulingLocation() { scheduler.newInstanceAvailable(i3); // schedule something on an arbitrary instance - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(Collections.emptyList()))); + SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(new Instance[0]))); // figure out how we use the location hints - Instance first = s1.getInstance(); + Instance first = (Instance) s1.getOwner(); Instance second = first != i1 ? i1 : i2; Instance third = first == i3 ? i2 : i3; // something that needs to go to the first instance again - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(Collections.singletonList(s1.getInstance())))); - assertEquals(first, s2.getInstance()); + SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(s1.getTaskManagerLocation()))); + assertEquals(first, s2.getOwner()); // first or second --> second, because first is full - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(Arrays.asList(first, second)))); - assertEquals(second, s3.getInstance()); + SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(first, second))); + assertEquals(second, s3.getOwner()); // first or third --> third (because first is full) - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(Arrays.asList(first, third)))); - SimpleSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(Arrays.asList(first, third)))); - assertEquals(third, s4.getInstance()); - assertEquals(third, s5.getInstance()); + SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(first, third))); + SimpleSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(first, third))); + assertEquals(third, s4.getOwner()); + assertEquals(third, s5.getOwner()); // first or third --> second, because all others are full - SimpleSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(Arrays.asList(first, third)))); - assertEquals(second, s6.getInstance()); + SimpleSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(first, third))); + assertEquals(second, s6.getOwner()); // release something on the first and second instance s2.releaseSlot(); s6.releaseSlot(); - SimpleSlot s7 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(Arrays.asList(first, third)))); - assertEquals(first, s7.getInstance()); + SimpleSlot s7 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(first, third))); + assertEquals(first, s7.getOwner()); assertEquals(1, scheduler.getNumberOfUnconstrainedAssignments()); assertEquals(1, scheduler.getNumberOfNonLocalizedAssignments()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java index 3e90123b4e7c8..a683834553d32 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java @@ -18,12 +18,13 @@ package org.apache.flink.runtime.jobmanager.scheduler; -import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.areAllDistinct; -import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertex; -import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertexWithLocation; -import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getRandomInstance; -import static org.apache.flink.runtime.testutils.CommonTestUtils.sleepUninterruptibly; -import static org.junit.Assert.*; +import org.apache.flink.runtime.instance.Instance; +import org.apache.flink.runtime.instance.SimpleSlot; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.runtime.testingUtils.TestingUtils; + +import org.junit.Test; import java.util.Random; import java.util.concurrent.ExecutorService; @@ -31,11 +32,16 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.flink.runtime.instance.SimpleSlot; -import org.apache.flink.runtime.testingUtils.TestingUtils; -import org.junit.Test; -import org.apache.flink.runtime.instance.Instance; -import org.apache.flink.runtime.jobgraph.JobVertexID; +import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.areAllDistinct; +import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getRandomInstance; +import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertex; +import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertexWithLocation; +import static org.apache.flink.runtime.testutils.CommonTestUtils.sleepUninterruptibly; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; /** * Tests for the scheduler when scheduling tasks in slot sharing groups. @@ -102,10 +108,10 @@ public void scheduleSingleVertexType() { // make sure we have two slots on the first instance, and two on the second int c = 0; - c += (s5.getInstance() == i1) ? 1 : -1; - c += (s6.getInstance() == i1) ? 1 : -1; - c += (s7.getInstance() == i1) ? 1 : -1; - c += (s8.getInstance() == i1) ? 1 : -1; + c += (s5.getTaskManagerID().equals(i1.getResourceId())) ? 1 : -1; + c += (s6.getTaskManagerID().equals(i1.getResourceId())) ? 1 : -1; + c += (s7.getTaskManagerID().equals(i1.getResourceId())) ? 1 : -1; + c += (s8.getTaskManagerID().equals(i1.getResourceId())) ? 1 : -1; assertEquals(0, c); // release all @@ -625,20 +631,23 @@ public void testLocalizedAssignment1() { try { JobVertexID jid1 = new JobVertexID(); JobVertexID jid2 = new JobVertexID(); - + SlotSharingGroup sharingGroup = new SlotSharingGroup(jid1, jid2); - + Instance i1 = getRandomInstance(2); Instance i2 = getRandomInstance(2); - + + TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); + TaskManagerLocation loc2 = i2.getInstanceConnectionInfo(); + Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext()); scheduler.newInstanceAvailable(i1); scheduler.newInstanceAvailable(i2); // schedule one to each instance - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, i1), sharingGroup)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, i2), sharingGroup)); + SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup)); + SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup)); assertNotNull(s1); assertNotNull(s2); @@ -647,8 +656,8 @@ public void testLocalizedAssignment1() { assertEquals(1, i2.getNumberOfAvailableSlots()); // schedule one from the other group to each instance - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, i1), sharingGroup)); - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, i2), sharingGroup)); + SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup)); + SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc2), sharingGroup)); assertNotNull(s3); assertNotNull(s4); @@ -675,20 +684,23 @@ public void testLocalizedAssignment2() { try { JobVertexID jid1 = new JobVertexID(); JobVertexID jid2 = new JobVertexID(); - + SlotSharingGroup sharingGroup = new SlotSharingGroup(jid1, jid2); - + Instance i1 = getRandomInstance(2); Instance i2 = getRandomInstance(2); - + + TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); + TaskManagerLocation loc2 = i2.getInstanceConnectionInfo(); + Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext()); scheduler.newInstanceAvailable(i1); scheduler.newInstanceAvailable(i2); // schedule one to each instance - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, i1), sharingGroup)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, i1), sharingGroup)); + SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup)); + SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc1), sharingGroup)); assertNotNull(s1); assertNotNull(s2); @@ -697,8 +709,8 @@ public void testLocalizedAssignment2() { assertEquals(2, i2.getNumberOfAvailableSlots()); // schedule one from the other group to each instance - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, i2), sharingGroup)); - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, i2), sharingGroup)); + SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc2), sharingGroup)); + SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc2), sharingGroup)); assertNotNull(s3); assertNotNull(s4); @@ -725,25 +737,27 @@ public void testLocalizedAssignment3() { try { JobVertexID jid1 = new JobVertexID(); JobVertexID jid2 = new JobVertexID(); - + SlotSharingGroup sharingGroup = new SlotSharingGroup(jid1, jid2); - + Instance i1 = getRandomInstance(2); Instance i2 = getRandomInstance(2); - + + TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); + Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext()); scheduler.newInstanceAvailable(i1); scheduler.newInstanceAvailable(i2); // schedule until the one instance is full - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, i1), sharingGroup)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, i1), sharingGroup)); - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 4, i1), sharingGroup)); - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 4, i1), sharingGroup)); + SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup)); + SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc1), sharingGroup)); + SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 4, loc1), sharingGroup)); + SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 4, loc1), sharingGroup)); // schedule two more with preference of same instance --> need to go to other instance - SimpleSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 3, 4, i1), sharingGroup)); - SimpleSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 4, 4, i1), sharingGroup)); + SimpleSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 3, 4, loc1), sharingGroup)); + SimpleSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 4, 4, loc1), sharingGroup)); assertNotNull(s1); assertNotNull(s2); @@ -757,12 +771,12 @@ public void testLocalizedAssignment3() { assertEquals(0, i1.getNumberOfAvailableSlots()); assertEquals(0, i2.getNumberOfAvailableSlots()); - assertEquals(i1, s1.getInstance()); - assertEquals(i1, s2.getInstance()); - assertEquals(i1, s3.getInstance()); - assertEquals(i1, s4.getInstance()); - assertEquals(i2, s5.getInstance()); - assertEquals(i2, s6.getInstance()); + assertEquals(i1.getResourceId(), s1.getTaskManagerID()); + assertEquals(i1.getResourceId(), s2.getTaskManagerID()); + assertEquals(i1.getResourceId(), s3.getTaskManagerID()); + assertEquals(i1.getResourceId(), s4.getTaskManagerID()); + assertEquals(i2.getResourceId(), s5.getTaskManagerID()); + assertEquals(i2.getResourceId(), s6.getTaskManagerID()); // check the scheduler's bookkeeping assertEquals(4, scheduler.getNumberOfLocalizedAssignments()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java index 99360e24f012f..eef27a8d73a8f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java @@ -23,9 +23,11 @@ import java.net.InetAddress; import java.net.UnknownHostException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashSet; +import java.util.List; import java.util.concurrent.atomic.AtomicInteger; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -82,8 +84,21 @@ public static Execution getDummyTask() { return execution; } + + public static Execution getTestVertex(Instance... preferredInstances) { + List locations = new ArrayList<>(preferredInstances.length); + for (Instance i : preferredInstances) { + locations.add(i.getInstanceConnectionInfo()); + } + return getTestVertex(locations); + } + + public static Execution getTestVertex(TaskManagerLocation... preferredLocations) { + return getTestVertex(Arrays.asList(preferredLocations)); + } + - public static Execution getTestVertex(Iterable preferredLocations) { + public static Execution getTestVertex(Iterable preferredLocations) { ExecutionVertex vertex = mock(ExecutionVertex.class); when(vertex.getPreferredLocations()).thenReturn(preferredLocations); @@ -113,10 +128,12 @@ public static Execution getTestVertex(JobVertexID jid, int taskIndex, int numTas return execution; } - - public static Execution getTestVertexWithLocation(JobVertexID jid, int taskIndex, int numTasks, Instance... locations) { + + public static Execution getTestVertexWithLocation( + JobVertexID jid, int taskIndex, int numTasks, TaskManagerLocation... locations) { + ExecutionVertex vertex = mock(ExecutionVertex.class); - + when(vertex.getPreferredLocations()).thenReturn(Arrays.asList(locations)); when(vertex.getJobId()).thenReturn(new JobID()); when(vertex.getJobvertexId()).thenReturn(jid); @@ -124,10 +141,10 @@ public static Execution getTestVertexWithLocation(JobVertexID jid, int taskIndex when(vertex.getTotalNumberOfParallelSubtasks()).thenReturn(numTasks); when(vertex.getMaxParallelism()).thenReturn(numTasks); when(vertex.toString()).thenReturn("TEST-VERTEX"); - + Execution execution = mock(Execution.class); when(execution.getVertex()).thenReturn(vertex); - + return execution; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureTest.java index d678531da4602..d9c100c080ed3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureTest.java @@ -18,14 +18,19 @@ package org.apache.flink.runtime.jobmanager.scheduler; -import static org.junit.Assert.*; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.instance.Instance; +import org.apache.flink.runtime.instance.SimpleSlot; + +import org.junit.Test; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.flink.runtime.instance.SimpleSlot; -import org.apache.flink.api.common.JobID; -import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.fail; public class SlotAllocationFutureTest { @@ -46,9 +51,14 @@ public void slotAllocated(SimpleSlot slot) {} } catch (IllegalStateException e) { // expected } - - final SimpleSlot slot1 = new SimpleSlot(new JobID(), SchedulerTestUtils.getRandomInstance(1), 0, null, null); - final SimpleSlot slot2 = new SimpleSlot(new JobID(), SchedulerTestUtils.getRandomInstance(1), 0, null, null); + + final Instance instance1 = SchedulerTestUtils.getRandomInstance(1); + final Instance instance2 = SchedulerTestUtils.getRandomInstance(1); + + final SimpleSlot slot1 = new SimpleSlot(new JobID(), instance1, + instance1.getInstanceConnectionInfo(), 0, instance1.getActorGateway(), null, null); + final SimpleSlot slot2 = new SimpleSlot(new JobID(), instance2, + instance2.getInstanceConnectionInfo(), 0, instance2.getActorGateway(), null, null); future.setSlot(slot1); try { @@ -71,7 +81,11 @@ public void setWithAction() { // action before the slot { final AtomicInteger invocations = new AtomicInteger(); - final SimpleSlot thisSlot = new SimpleSlot(new JobID(), SchedulerTestUtils.getRandomInstance(1), 0, null, null); + + final Instance instance = SchedulerTestUtils.getRandomInstance(1); + + final SimpleSlot thisSlot = new SimpleSlot(new JobID(), instance, + instance.getInstanceConnectionInfo(), 0, instance.getActorGateway(), null, null); SlotAllocationFuture future = new SlotAllocationFuture(); @@ -91,7 +105,10 @@ public void slotAllocated(SimpleSlot slot) { // slot before action { final AtomicInteger invocations = new AtomicInteger(); - final SimpleSlot thisSlot = new SimpleSlot(new JobID(), SchedulerTestUtils.getRandomInstance(1), 0, null, null); + final Instance instance = SchedulerTestUtils.getRandomInstance(1); + + final SimpleSlot thisSlot = new SimpleSlot(new JobID(), instance, + instance.getInstanceConnectionInfo(), 0, instance.getActorGateway(), null, null); SlotAllocationFuture future = new SlotAllocationFuture(); future.setSlot(thisSlot); @@ -120,8 +137,11 @@ public void setSync() { { final AtomicInteger invocations = new AtomicInteger(); final AtomicBoolean error = new AtomicBoolean(); - - final SimpleSlot thisSlot = new SimpleSlot(new JobID(), SchedulerTestUtils.getRandomInstance(1), 0, null, null); + + final Instance instance = SchedulerTestUtils.getRandomInstance(1); + + final SimpleSlot thisSlot = new SimpleSlot(new JobID(), instance, + instance.getInstanceConnectionInfo(), 0, instance.getActorGateway(), null, null); final SlotAllocationFuture future = new SlotAllocationFuture(); @@ -130,7 +150,7 @@ public void setSync() { @Override public void run() { try { - SimpleSlot syncSlot = future.waitTillAllocated(); + SimpleSlot syncSlot = future.waitTillCompleted(); if (syncSlot == null || syncSlot != thisSlot) { error.set(true); return; @@ -158,12 +178,15 @@ public void run() { // setting slot before syncing { - final SimpleSlot thisSlot = new SimpleSlot(new JobID(), SchedulerTestUtils.getRandomInstance(1), 0, null, null); + final Instance instance = SchedulerTestUtils.getRandomInstance(1); + + final SimpleSlot thisSlot = new SimpleSlot(new JobID(), instance, + instance.getInstanceConnectionInfo(), 0, instance.getActorGateway(), null, null); final SlotAllocationFuture future = new SlotAllocationFuture(); future.setSlot(thisSlot); - SimpleSlot retrieved = future.waitTillAllocated(); + SimpleSlot retrieved = future.waitTillCompleted(); assertNotNull(retrieved); assertEquals(thisSlot, retrieved); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerITCase.java index 3307568bf99ef..0c2ca1a0aeef8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerITCase.java @@ -24,20 +24,24 @@ import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.instance.ActorGateway; +import org.apache.flink.runtime.instance.HardwareDescription; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.messages.Messages; import org.apache.flink.runtime.messages.RegistrationMessages; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testutils.TestingResourceManager; import org.apache.flink.util.TestLogger; + import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; -import org.mockito.Mockito; + import scala.Option; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; /** * It cases which test the interaction of the resource manager with job manager and task managers. @@ -79,12 +83,13 @@ protected void run() { ResourceID resourceID = ResourceID.generate(); + TaskManagerLocation location = mock(TaskManagerLocation.class); + when(location.getResourceID()).thenReturn(resourceID); + + HardwareDescription resourceProfile = HardwareDescription.extractFromSystem(1_000_000); + jobManager.tell( - new RegistrationMessages.RegisterTaskManager( - resourceID, - Mockito.mock(TaskManagerLocation.class), - null, - 1), + new RegistrationMessages.RegisterTaskManager(resourceID, location, resourceProfile, 1), me); expectMsgClass(RegistrationMessages.AcknowledgeRegistration.class); From 7fcea90507881eb4ef79e486b764c0c5f3210e21 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 31 Aug 2016 13:59:01 +0200 Subject: [PATCH 018/299] [FLINK-4490] [distributed coordination] (part 3) Rename methods on 'Instance' to have more intuitive names getResourceID() --> getTaskManagerID() getInstanceConnectionInfo() --> getTaskManagerLocation() --- .../handlers/TaskManagersHandler.java | 2 +- ...rtialInputChannelDeploymentDescriptor.java | 21 +- .../flink/runtime/instance/Instance.java | 28 +- .../runtime/instance/InstanceManager.java | 20 +- .../jobmanager/scheduler/Scheduler.java | 18 +- .../flink/runtime/jobmanager/JobManager.scala | 3 +- .../ExecutionGraphMetricsTest.java | 404 +++++++++--------- .../ExecutionGraphTestUtils.java | 2 +- .../TerminalStateDeadlockTest.java | 3 +- .../runtime/instance/InstanceManagerTest.java | 31 +- .../flink/runtime/instance/InstanceTest.java | 6 +- .../runtime/instance/SharedSlotsTest.java | 24 +- .../runtime/instance/SimpleSlotTest.java | 2 +- .../SpilledSubpartitionViewTest.java | 4 +- .../scheduler/CoLocationConstraintTest.java | 6 +- .../ScheduleWithCoLocationHintTest.java | 22 +- .../scheduler/SchedulerSlotSharingTest.java | 30 +- .../scheduler/SchedulerTestUtils.java | 5 +- .../scheduler/SlotAllocationFutureTest.java | 12 +- 19 files changed, 319 insertions(+), 324 deletions(-) diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandler.java index b60cd105b1fe3..b5e90888bf146 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandler.java @@ -85,7 +85,7 @@ public String handleRequest(Map pathParams, Map gen.writeStartObject(); gen.writeStringField("id", instance.getId().toString()); gen.writeStringField("path", instance.getActorGateway().path()); - gen.writeNumberField("dataPort", instance.getInstanceConnectionInfo().dataPort()); + gen.writeNumberField("dataPort", instance.getTaskManagerLocation().dataPort()); gen.writeNumberField("timeSinceLastHeartbeat", instance.getLastHeartBeat()); gen.writeNumberField("slotsNumber", instance.getTotalNumberOfSlots()); gen.writeNumberField("freeSlots", instance.getNumberOfAvailableSlots()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartialInputChannelDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartialInputChannelDeploymentDescriptor.java index e1391a46d7ea2..0eac39d2f7d1a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartialInputChannelDeploymentDescriptor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartialInputChannelDeploymentDescriptor.java @@ -44,7 +44,7 @@ public class PartialInputChannelDeploymentDescriptor { private final ResultPartitionID partitionID; /** The partition connection info. */ - private final TaskManagerLocation partitionConnectionInfo; + private final TaskManagerLocation partitionTaskManagerLocation; /** The partition connection index. */ private final int partitionConnectionIndex; @@ -52,12 +52,12 @@ public class PartialInputChannelDeploymentDescriptor { public PartialInputChannelDeploymentDescriptor( IntermediateDataSetID resultId, ResultPartitionID partitionID, - TaskManagerLocation partitionConnectionInfo, + TaskManagerLocation partitionTaskManagerLocation, int partitionConnectionIndex) { this.resultId = checkNotNull(resultId); this.partitionID = checkNotNull(partitionID); - this.partitionConnectionInfo = checkNotNull(partitionConnectionInfo); + this.partitionTaskManagerLocation = checkNotNull(partitionTaskManagerLocation); this.partitionConnectionIndex = partitionConnectionIndex; } @@ -66,23 +66,20 @@ public PartialInputChannelDeploymentDescriptor( * * @see InputChannelDeploymentDescriptor */ - public InputChannelDeploymentDescriptor createInputChannelDeploymentDescriptor( - Execution consumerExecution) { + public InputChannelDeploymentDescriptor createInputChannelDeploymentDescriptor(Execution consumerExecution) { + checkNotNull(consumerExecution, "consumerExecution"); - checkNotNull(consumerExecution, "Consumer execution null"); - - TaskManagerLocation consumerConnectionInfo = consumerExecution.getAssignedResourceLocation(); - - checkNotNull(consumerConnectionInfo, "Consumer connection info null"); + TaskManagerLocation consumerLocation = consumerExecution.getAssignedResourceLocation(); + checkNotNull(consumerLocation, "Consumer connection info null"); final ResultPartitionLocation partitionLocation; - if (consumerConnectionInfo.equals(partitionConnectionInfo)) { + if (consumerLocation.equals(partitionTaskManagerLocation)) { partitionLocation = ResultPartitionLocation.createLocal(); } else { partitionLocation = ResultPartitionLocation.createRemote( - new ConnectionID(partitionConnectionInfo, partitionConnectionIndex)); + new ConnectionID(partitionTaskManagerLocation, partitionConnectionIndex)); } return new InputChannelDeploymentDescriptor(partitionID, partitionLocation); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java index fe46895c4500b..4a8139b208581 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java @@ -52,14 +52,11 @@ public class Instance implements SlotOwner { private final ActorGateway actorGateway; /** The instance connection information for the data transfer. */ - private final TaskManagerLocation connectionInfo; + private final TaskManagerLocation location; /** A description of the resources of the task manager */ private final HardwareDescription resources; - /** The ID identifies the resource the task manager runs on */ - private final ResourceID resourceId; - /** The ID identifying the taskManager. */ private final InstanceID instanceId; @@ -90,22 +87,19 @@ public class Instance implements SlotOwner { * Constructs an instance reflecting a registered TaskManager. * * @param actorGateway The actor gateway to communicate with the remote instance - * @param connectionInfo The remote connection where the task manager receives requests. - * @param resourceId The resource id which denotes the resource the task manager uses. + * @param location The remote connection where the task manager receives requests. * @param id The id under which the taskManager is registered. * @param resources The resources available on the machine. * @param numberOfSlots The number of task slots offered by this taskManager. */ public Instance( ActorGateway actorGateway, - TaskManagerLocation connectionInfo, - ResourceID resourceId, + TaskManagerLocation location, InstanceID id, HardwareDescription resources, int numberOfSlots) { this.actorGateway = actorGateway; - this.connectionInfo = connectionInfo; - this.resourceId = resourceId; + this.location = location; this.instanceId = id; this.resources = resources; this.numberOfSlots = numberOfSlots; @@ -120,8 +114,8 @@ public Instance( // Properties // -------------------------------------------------------------------------------------------- - public ResourceID getResourceId() { - return resourceId; + public ResourceID getTaskManagerID() { + return location.getResourceID(); } public InstanceID getId() { @@ -246,7 +240,7 @@ public SimpleSlot allocateSimpleSlot(JobID jobID) throws InstanceDiedException { return null; } else { - SimpleSlot slot = new SimpleSlot(jobID, this, connectionInfo, nextSlot, actorGateway); + SimpleSlot slot = new SimpleSlot(jobID, this, location, nextSlot, actorGateway); allocatedSlots.add(slot); return slot; } @@ -284,7 +278,7 @@ public SharedSlot allocateSharedSlot(JobID jobID, SlotSharingGroupAssignment sha } else { SharedSlot slot = new SharedSlot( - jobID, this, connectionInfo, nextSlot, actorGateway, sharingGroupAssignment); + jobID, this, location, nextSlot, actorGateway, sharingGroupAssignment); allocatedSlots.add(slot); return slot; } @@ -355,8 +349,8 @@ public ActorGateway getActorGateway() { return actorGateway; } - public TaskManagerLocation getInstanceConnectionInfo() { - return connectionInfo; + public TaskManagerLocation getTaskManagerLocation() { + return location; } public int getNumberOfAvailableSlots() { @@ -405,7 +399,7 @@ public void removeSlotListener() { @Override public String toString() { - return String.format("%s @ %s - %d slots - URL: %s", instanceId, connectionInfo.getHostname(), + return String.format("%s @ %s - %d slots - URL: %s", instanceId, location.getHostname(), numberOfSlots, (actorGateway != null ? actorGateway.path() : "No instance gateway")); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java index e7a45376d4631..0c7e18783b2cd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java @@ -138,8 +138,7 @@ public boolean reportHeartBeat(InstanceID instanceId, byte[] lastMetricsReport) * for the job execution. * * @param taskManager ActorRef to the TaskManager which wants to be registered - * @param resourceID The resource id of the TaskManager - * @param connectionInfo ConnectionInfo of the TaskManager + * @param taskManagerLocation Location info of the TaskManager * @param resources Hardware description of the TaskManager * @param numberOfSlots Number of available slots on the TaskManager * @param leaderSessionID The current leader session ID of the JobManager @@ -147,12 +146,12 @@ public boolean reportHeartBeat(InstanceID instanceId, byte[] lastMetricsReport) */ public InstanceID registerTaskManager( ActorRef taskManager, - ResourceID resourceID, - TaskManagerLocation connectionInfo, + TaskManagerLocation taskManagerLocation, HardwareDescription resources, int numberOfSlots, - UUID leaderSessionID){ - synchronized(this.lock){ + UUID leaderSessionID) { + + synchronized (this.lock) { if (this.isShutdown) { throw new IllegalStateException("InstanceManager is shut down."); } @@ -174,12 +173,11 @@ public InstanceID registerTaskManager( InstanceID instanceID = new InstanceID(); - Instance host = new Instance(actorGateway, connectionInfo, resourceID, instanceID, - resources, numberOfSlots); + Instance host = new Instance(actorGateway, taskManagerLocation, instanceID, resources, numberOfSlots); registeredHostsById.put(instanceID, host); registeredHostsByConnection.put(taskManager, host); - registeredHostsByResource.put(resourceID, host); + registeredHostsByResource.put(taskManagerLocation.getResourceID(), host); totalNumberOfAliveTaskSlots += numberOfSlots; @@ -187,7 +185,7 @@ public InstanceID registerTaskManager( LOG.info(String.format("Registered TaskManager at %s (%s) as %s. " + "Current number of registered hosts is %d. " + "Current number of alive task slots is %d.", - connectionInfo.getHostname(), + taskManagerLocation.getHostname(), taskManager.path(), instanceID, registeredHostsById.size(), @@ -217,7 +215,7 @@ public void unregisterTaskManager(ActorRef instanceID, boolean terminated){ registeredHostsByConnection.remove(host); registeredHostsById.remove(instance.getId()); - registeredHostsByResource.remove(instance.getResourceId()); + registeredHostsByResource.remove(instance.getTaskManagerID()); if (terminated) { deadHosts.add(instance.getActorGateway().actor()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java index b481b55ec12b4..734972dd8dc72 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java @@ -363,7 +363,7 @@ protected SimpleSlot getFreeSlotForTask(ExecutionVertex vertex, // if the instance has further available slots, re-add it to the set of available resources. if (instanceToUse.hasResourcesAvailable()) { - this.instancesWithAvailableResources.put(instanceToUse.getResourceId(), instanceToUse); + this.instancesWithAvailableResources.put(instanceToUse.getTaskManagerID(), instanceToUse); } if (slot != null) { @@ -425,7 +425,7 @@ protected SimpleSlot getNewSlotForSharingGroup(ExecutionVertex vertex, // if the instance has further available slots, re-add it to the set of available resources. if (instanceToUse.hasResourcesAvailable()) { - this.instancesWithAvailableResources.put(instanceToUse.getResourceId(), instanceToUse); + this.instancesWithAvailableResources.put(instanceToUse.getTaskManagerID(), instanceToUse); } if (sharedSlot != null) { @@ -469,7 +469,7 @@ private Pair findInstance(Iterable requ while (this.newlyAvailableInstances.size() > 0) { Instance queuedInstance = this.newlyAvailableInstances.poll(); if (queuedInstance != null) { - this.instancesWithAvailableResources.put(queuedInstance.getResourceId(), queuedInstance); + this.instancesWithAvailableResources.put(queuedInstance.getTaskManagerID(), queuedInstance); } } @@ -583,7 +583,7 @@ private void handleNewSlot() { } } else { - this.instancesWithAvailableResources.put(instance.getResourceId(), instance); + this.instancesWithAvailableResources.put(instance.getTaskManagerID(), instance); } } } @@ -649,7 +649,7 @@ public void newInstanceAvailable(Instance instance) { instance.setSlotAvailabilityListener(this); // store the instance in the by-host-lookup - String instanceHostName = instance.getInstanceConnectionInfo().getHostname(); + String instanceHostName = instance.getTaskManagerLocation().getHostname(); Set instanceSet = allInstancesByHost.get(instanceHostName); if (instanceSet == null) { instanceSet = new HashSet(); @@ -658,7 +658,7 @@ public void newInstanceAvailable(Instance instance) { instanceSet.add(instance); // add it to the available resources and let potential waiters know - this.instancesWithAvailableResources.put(instance.getResourceId(), instance); + this.instancesWithAvailableResources.put(instance.getTaskManagerID(), instance); // add all slots as available for (int i = 0; i < instance.getNumberOfAvailableSlots(); i++) { @@ -693,9 +693,9 @@ private void removeInstance(Instance instance) { } allInstances.remove(instance); - instancesWithAvailableResources.remove(instance.getResourceId()); + instancesWithAvailableResources.remove(instance.getTaskManagerID()); - String instanceHostName = instance.getInstanceConnectionInfo().getHostname(); + String instanceHostName = instance.getTaskManagerLocation().getHostname(); Set instanceSet = allInstancesByHost.get(instanceHostName); if (instanceSet != null) { instanceSet.remove(instance); @@ -795,7 +795,7 @@ private void processNewlyAvailableInstances() { while ((instance = newlyAvailableInstances.poll()) != null) { if (instance.hasResourcesAvailable()) { - instancesWithAvailableResources.put(instance.getResourceId(), instance); + instancesWithAvailableResources.put(instance.getTaskManagerID(), instance); } } } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index 2a0ecc2e36545..88af604504f29 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -349,7 +349,7 @@ class JobManager( currentResourceManager = Option(msg.resourceManager()) val taskManagerResources = instanceManager.getAllRegisteredInstances.asScala.map( - instance => instance.getResourceId).toList.asJava + instance => instance.getTaskManagerID).toList.asJava // confirm registration and send known task managers with their resource ids sender ! decorateMessage(new RegisterResourceManagerSuccessful(self, taskManagerResources)) @@ -425,7 +425,6 @@ class JobManager( try { val instanceID = instanceManager.registerTaskManager( taskManager, - resourceId, connectionInfo, hardwareInformation, numberOfSlots, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java index d8bd6cbe9cad5..d5520fddaf590 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java @@ -28,6 +28,7 @@ import org.apache.flink.metrics.reporter.MetricReporter; import org.apache.flink.runtime.JobException; import org.apache.flink.runtime.blob.BlobKey; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; import org.apache.flink.runtime.instance.ActorGateway; @@ -47,8 +48,10 @@ import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.util.SerializedValue; import org.apache.flink.util.TestLogger; + import org.junit.Test; import org.mockito.Matchers; + import scala.concurrent.ExecutionContext$; import scala.concurrent.Future$; import scala.concurrent.duration.FiniteDuration; @@ -60,7 +63,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import static org.junit.Assert.assertEquals; @@ -76,200 +80,210 @@ public class ExecutionGraphMetricsTest extends TestLogger { */ @Test public void testExecutionGraphRestartTimeMetric() throws JobException, IOException, InterruptedException { - // setup execution graph with mocked scheduling logic - int parallelism = 1; - - JobVertex jobVertex = new JobVertex("TestVertex"); - jobVertex.setParallelism(parallelism); - jobVertex.setInvokableClass(Tasks.NoOpInvokable.class); - JobGraph jobGraph = new JobGraph("Test Job", jobVertex); - - Configuration config = new Configuration(); - config.setString(ConfigConstants.METRICS_REPORTERS_LIST, "test"); - config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestingReporter.class.getName()); - - Configuration jobConfig = new Configuration(); - - FiniteDuration timeout = new FiniteDuration(10, TimeUnit.SECONDS); - - MetricRegistry metricRegistry = new MetricRegistry(config); - - assertTrue(metricRegistry.getReporters().size() == 1); - - MetricReporter reporter = metricRegistry.getReporters().get(0); - - assertTrue(reporter instanceof TestingReporter); - - TestingReporter testingReporter = (TestingReporter) reporter; - - MetricGroup metricGroup = new JobManagerMetricGroup(metricRegistry, "localhost"); - - Scheduler scheduler = mock(Scheduler.class); - - SimpleSlot simpleSlot = mock(SimpleSlot.class); - - Instance instance = mock(Instance.class); - - TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class); - - Slot rootSlot = mock(Slot.class); - - ActorGateway actorGateway = mock(ActorGateway.class); - - when(simpleSlot.isAlive()).thenReturn(true); - when(simpleSlot.getTaskManagerID()).thenReturn(instance.getResourceId()); - when(simpleSlot.getTaskManagerLocation()).thenReturn(instance.getInstanceConnectionInfo()); - when(simpleSlot.setExecutedVertex(Matchers.any(Execution.class))).thenReturn(true); - when(simpleSlot.getRoot()).thenReturn(rootSlot); - - when(scheduler.scheduleImmediately(Matchers.any(ScheduledUnit.class))).thenReturn(simpleSlot); - - when(instance.getInstanceConnectionInfo()).thenReturn(taskManagerLocation); - when(instance.getActorGateway()).thenReturn(actorGateway); - when(taskManagerLocation.getHostname()).thenReturn("localhost"); - - when(rootSlot.getSlotNumber()).thenReturn(0); - - when(actorGateway.ask(Matchers.any(Object.class), Matchers.any(FiniteDuration.class))).thenReturn(Future$.MODULE$.successful(Messages.getAcknowledge())); - - TestingRestartStrategy testingRestartStrategy = new TestingRestartStrategy(); - - ExecutionGraph executionGraph = new ExecutionGraph( - ExecutionContext$.MODULE$.fromExecutor(new ForkJoinPool()), - jobGraph.getJobID(), - jobGraph.getName(), - jobConfig, - new SerializedValue(null), - timeout, - testingRestartStrategy, - Collections.emptyList(), - Collections.emptyList(), - getClass().getClassLoader(), - metricGroup); - - // get restarting time metric - Metric metric = testingReporter.getMetric(ExecutionGraph.RESTARTING_TIME_METRIC_NAME); - - assertNotNull(metric); - assertTrue(metric instanceof Gauge); - - @SuppressWarnings("unchecked") - Gauge restartingTime = (Gauge) metric; - - // check that the restarting time is 0 since it's the initial start - assertTrue(0L == restartingTime.getValue()); - - executionGraph.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources()); - - // start execution - executionGraph.scheduleForExecution(scheduler); - - assertTrue(0L == restartingTime.getValue()); - - List executionIDs = new ArrayList<>(); - - for (ExecutionVertex executionVertex: executionGraph.getAllExecutionVertices()) { - executionIDs.add(executionVertex.getCurrentExecutionAttempt().getAttemptId()); - } - - // tell execution graph that the tasks are in state running --> job status switches to state running - for (ExecutionAttemptID executionID : executionIDs) { - executionGraph.updateState(new TaskExecutionState(jobGraph.getJobID(), executionID, ExecutionState.RUNNING)); - } - - assertEquals(JobStatus.RUNNING, executionGraph.getState()); - - assertTrue(0L == restartingTime.getValue()); - - // fail the job so that it goes into state restarting - for (ExecutionAttemptID executionID : executionIDs) { - executionGraph.updateState(new TaskExecutionState(jobGraph.getJobID(), executionID, ExecutionState.FAILED, new Exception())); - } - - assertEquals(JobStatus.RESTARTING, executionGraph.getState()); - - long firstRestartingTimestamp = executionGraph.getStatusTimestamp(JobStatus.RESTARTING); - - // wait some time so that the restarting time gauge shows a value different from 0 - Thread.sleep(50); - - long previousRestartingTime = restartingTime.getValue(); - - // check that the restarting time is monotonically increasing - for (int i = 0; i < 10; i++) { - long currentRestartingTime = restartingTime.getValue(); - - assertTrue(currentRestartingTime >= previousRestartingTime); - previousRestartingTime = currentRestartingTime; - } - - // check that we have measured some restarting time - assertTrue(previousRestartingTime > 0); - - // restart job - testingRestartStrategy.restartExecutionGraph(); - - executionIDs.clear(); - - for (ExecutionVertex executionVertex: executionGraph.getAllExecutionVertices()) { - executionIDs.add(executionVertex.getCurrentExecutionAttempt().getAttemptId()); - } - - for (ExecutionAttemptID executionID : executionIDs) { - executionGraph.updateState(new TaskExecutionState(jobGraph.getJobID(), executionID, ExecutionState.RUNNING)); - } - - assertEquals(JobStatus.RUNNING, executionGraph.getState()); - - assertTrue(firstRestartingTimestamp != 0); - - previousRestartingTime = restartingTime.getValue(); - - // check that the restarting time does not increase after we've reached the running state - for (int i = 0; i < 10; i++) { - long currentRestartingTime = restartingTime.getValue(); - - assertTrue(currentRestartingTime == previousRestartingTime); - previousRestartingTime = currentRestartingTime; - } - - // fail job again - for (ExecutionAttemptID executionID : executionIDs) { - executionGraph.updateState(new TaskExecutionState(jobGraph.getJobID(), executionID, ExecutionState.FAILED, new Exception())); - } - - assertEquals(JobStatus.RESTARTING, executionGraph.getState()); - - long secondRestartingTimestamp = executionGraph.getStatusTimestamp(JobStatus.RESTARTING); - - assertTrue(firstRestartingTimestamp != secondRestartingTimestamp); - - Thread.sleep(50); - - previousRestartingTime = restartingTime.getValue(); - - // check that the restarting time is increasing again - for (int i = 0; i < 10; i++) { - long currentRestartingTime = restartingTime.getValue(); - - assertTrue(currentRestartingTime >= previousRestartingTime); - previousRestartingTime = currentRestartingTime; - } - - assertTrue(previousRestartingTime > 0); - - // now lets fail the job while it is in restarting and see whether the restarting time then stops to increase - executionGraph.fail(new Exception()); - - assertEquals(JobStatus.FAILED, executionGraph.getState()); - - previousRestartingTime = restartingTime.getValue(); - - for (int i = 0; i < 10; i++) { - long currentRestartingTime = restartingTime.getValue(); - - assertTrue(currentRestartingTime == previousRestartingTime); - previousRestartingTime = currentRestartingTime; + final ExecutorService executor = Executors.newCachedThreadPool(); + try { + // setup execution graph with mocked scheduling logic + int parallelism = 1; + + JobVertex jobVertex = new JobVertex("TestVertex"); + jobVertex.setParallelism(parallelism); + jobVertex.setInvokableClass(Tasks.NoOpInvokable.class); + JobGraph jobGraph = new JobGraph("Test Job", jobVertex); + + Configuration config = new Configuration(); + config.setString(ConfigConstants.METRICS_REPORTERS_LIST, "test"); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestingReporter.class.getName()); + + Configuration jobConfig = new Configuration(); + + FiniteDuration timeout = new FiniteDuration(10, TimeUnit.SECONDS); + + MetricRegistry metricRegistry = new MetricRegistry(config); + + assertTrue(metricRegistry.getReporters().size() == 1); + + MetricReporter reporter = metricRegistry.getReporters().get(0); + + assertTrue(reporter instanceof TestingReporter); + + TestingReporter testingReporter = (TestingReporter) reporter; + + MetricGroup metricGroup = new JobManagerMetricGroup(metricRegistry, "localhost"); + + Scheduler scheduler = mock(Scheduler.class); + + ResourceID taskManagerId = ResourceID.generate(); + + TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class); + when(taskManagerLocation.getResourceID()).thenReturn(taskManagerId); + when(taskManagerLocation.getHostname()).thenReturn("localhost"); + + ActorGateway actorGateway = mock(ActorGateway.class); + + Instance instance = mock(Instance.class); + when(instance.getTaskManagerLocation()).thenReturn(taskManagerLocation); + when(instance.getTaskManagerID()).thenReturn(taskManagerId); + when(instance.getActorGateway()).thenReturn(actorGateway); + + Slot rootSlot = mock(Slot.class); + + SimpleSlot simpleSlot = mock(SimpleSlot.class); + when(simpleSlot.isAlive()).thenReturn(true); + when(simpleSlot.getTaskManagerLocation()).thenReturn(taskManagerLocation); + when(simpleSlot.getTaskManagerID()).thenReturn(taskManagerId); + when(simpleSlot.getTaskManagerActorGateway()).thenReturn(actorGateway); + when(simpleSlot.setExecutedVertex(Matchers.any(Execution.class))).thenReturn(true); + when(simpleSlot.getRoot()).thenReturn(rootSlot); + + when(scheduler.scheduleImmediately(Matchers.any(ScheduledUnit.class))).thenReturn(simpleSlot); + + + + when(rootSlot.getSlotNumber()).thenReturn(0); + + when(actorGateway.ask(Matchers.any(Object.class), Matchers.any(FiniteDuration.class))).thenReturn(Future$.MODULE$.successful(Messages.getAcknowledge())); + + TestingRestartStrategy testingRestartStrategy = new TestingRestartStrategy(); + + ExecutionGraph executionGraph = new ExecutionGraph( + ExecutionContext$.MODULE$.fromExecutor(executor), + jobGraph.getJobID(), + jobGraph.getName(), + jobConfig, + new SerializedValue(null), + timeout, + testingRestartStrategy, + Collections.emptyList(), + Collections.emptyList(), + getClass().getClassLoader(), + metricGroup); + + // get restarting time metric + Metric metric = testingReporter.getMetric(ExecutionGraph.RESTARTING_TIME_METRIC_NAME); + + assertNotNull(metric); + assertTrue(metric instanceof Gauge); + + @SuppressWarnings("unchecked") + Gauge restartingTime = (Gauge) metric; + + // check that the restarting time is 0 since it's the initial start + assertTrue(0L == restartingTime.getValue()); + + executionGraph.attachJobGraph(jobGraph.getVerticesSortedTopologicallyFromSources()); + + // start execution + executionGraph.scheduleForExecution(scheduler); + + assertTrue(0L == restartingTime.getValue()); + + List executionIDs = new ArrayList<>(); + + for (ExecutionVertex executionVertex: executionGraph.getAllExecutionVertices()) { + executionIDs.add(executionVertex.getCurrentExecutionAttempt().getAttemptId()); + } + + // tell execution graph that the tasks are in state running --> job status switches to state running + for (ExecutionAttemptID executionID : executionIDs) { + executionGraph.updateState(new TaskExecutionState(jobGraph.getJobID(), executionID, ExecutionState.RUNNING)); + } + + assertEquals(JobStatus.RUNNING, executionGraph.getState()); + + assertTrue(0L == restartingTime.getValue()); + + // fail the job so that it goes into state restarting + for (ExecutionAttemptID executionID : executionIDs) { + executionGraph.updateState(new TaskExecutionState(jobGraph.getJobID(), executionID, ExecutionState.FAILED, new Exception())); + } + + assertEquals(JobStatus.RESTARTING, executionGraph.getState()); + + long firstRestartingTimestamp = executionGraph.getStatusTimestamp(JobStatus.RESTARTING); + + // wait some time so that the restarting time gauge shows a value different from 0 + Thread.sleep(50); + + long previousRestartingTime = restartingTime.getValue(); + + // check that the restarting time is monotonically increasing + for (int i = 0; i < 10; i++) { + long currentRestartingTime = restartingTime.getValue(); + + assertTrue(currentRestartingTime >= previousRestartingTime); + previousRestartingTime = currentRestartingTime; + } + + // check that we have measured some restarting time + assertTrue(previousRestartingTime > 0); + + // restart job + testingRestartStrategy.restartExecutionGraph(); + + executionIDs.clear(); + + for (ExecutionVertex executionVertex: executionGraph.getAllExecutionVertices()) { + executionIDs.add(executionVertex.getCurrentExecutionAttempt().getAttemptId()); + } + + for (ExecutionAttemptID executionID : executionIDs) { + executionGraph.updateState(new TaskExecutionState(jobGraph.getJobID(), executionID, ExecutionState.RUNNING)); + } + + assertEquals(JobStatus.RUNNING, executionGraph.getState()); + + assertTrue(firstRestartingTimestamp != 0); + + previousRestartingTime = restartingTime.getValue(); + + // check that the restarting time does not increase after we've reached the running state + for (int i = 0; i < 10; i++) { + long currentRestartingTime = restartingTime.getValue(); + + assertTrue(currentRestartingTime == previousRestartingTime); + previousRestartingTime = currentRestartingTime; + } + + // fail job again + for (ExecutionAttemptID executionID : executionIDs) { + executionGraph.updateState(new TaskExecutionState(jobGraph.getJobID(), executionID, ExecutionState.FAILED, new Exception())); + } + + assertEquals(JobStatus.RESTARTING, executionGraph.getState()); + + long secondRestartingTimestamp = executionGraph.getStatusTimestamp(JobStatus.RESTARTING); + + assertTrue(firstRestartingTimestamp != secondRestartingTimestamp); + + Thread.sleep(50); + + previousRestartingTime = restartingTime.getValue(); + + // check that the restarting time is increasing again + for (int i = 0; i < 10; i++) { + long currentRestartingTime = restartingTime.getValue(); + + assertTrue(currentRestartingTime >= previousRestartingTime); + previousRestartingTime = currentRestartingTime; + } + + assertTrue(previousRestartingTime > 0); + + // now lets fail the job while it is in restarting and see whether the restarting time then stops to increase + executionGraph.fail(new Exception()); + + assertEquals(JobStatus.FAILED, executionGraph.getState()); + + previousRestartingTime = restartingTime.getValue(); + + for (int i = 0; i < 10; i++) { + long currentRestartingTime = restartingTime.getValue(); + + assertTrue(currentRestartingTime == previousRestartingTime); + previousRestartingTime = currentRestartingTime; + } + } finally { + executor.shutdownNow(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java index cddb6cbf61502..df47c3a9d7c7d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java @@ -113,7 +113,7 @@ public static Instance getInstance(final ActorGateway gateway, final int numberO InetAddress address = InetAddress.getByName("127.0.0.1"); TaskManagerLocation connection = new TaskManagerLocation(resourceID, address, 10001); - return new Instance(gateway, connection, resourceID, new InstanceID(), hardwareDescription, numberOfSlots); + return new Instance(gateway, connection, new InstanceID(), hardwareDescription, numberOfSlots); } @SuppressWarnings("serial") diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java index a71faf62eedc8..870ae05bf3c1e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java @@ -84,8 +84,7 @@ public TerminalStateDeadlockTest() { TaskManagerLocation ci = new TaskManagerLocation(resourceId, address, 12345); HardwareDescription resources = new HardwareDescription(4, 4000000, 3000000, 2000000); - Instance instance = new Instance(DummyActorGateway.INSTANCE, ci, - resourceId, new InstanceID(), resources, 4); + Instance instance = new Instance(DummyActorGateway.INSTANCE, ci, new InstanceID(), resources, 4); this.resource = instance.allocateSimpleSlot(new JobID()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java index f1ed9601e8ec3..f3747c824625b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java @@ -87,12 +87,9 @@ public void testInstanceRegistering() { final JavaTestKit probe2 = new JavaTestKit(system); final JavaTestKit probe3 = new JavaTestKit(system); - cm.registerTaskManager(probe1.getRef(), resID1, - ici1, hardwareDescription, 1, leaderSessionID); - cm.registerTaskManager(probe2.getRef(), resID2, - ici2, hardwareDescription, 2, leaderSessionID); - cm.registerTaskManager(probe3.getRef(), resID3, - ici3, hardwareDescription, 5, leaderSessionID); + cm.registerTaskManager(probe1.getRef(), ici1, hardwareDescription, 1, leaderSessionID); + cm.registerTaskManager(probe2.getRef(), ici2, hardwareDescription, 2, leaderSessionID); + cm.registerTaskManager(probe3.getRef(), ici3, hardwareDescription, 5, leaderSessionID); assertEquals(3, cm.getNumberOfRegisteredTaskManagers()); assertEquals(8, cm.getTotalNumberOfSlots()); @@ -102,7 +99,7 @@ public void testInstanceRegistering() { HashSet(); for(Instance instance: instances){ - taskManagerLocations.add(instance.getInstanceConnectionInfo()); + taskManagerLocations.add(instance.getTaskManagerLocation()); } assertTrue(taskManagerLocations.contains(ici1)); @@ -133,14 +130,13 @@ public void testRegisteringAlreadyRegistered() { TaskManagerLocation ici = new TaskManagerLocation(resID1, address, dataPort); JavaTestKit probe = new JavaTestKit(system); - cm.registerTaskManager(probe.getRef(), resID1, - ici, resources, 1, leaderSessionID); + cm.registerTaskManager(probe.getRef(), ici, resources, 1, leaderSessionID); assertEquals(1, cm.getNumberOfRegisteredTaskManagers()); assertEquals(1, cm.getTotalNumberOfSlots()); try { - cm.registerTaskManager(probe.getRef(), resID2, ici, resources, 1, leaderSessionID); + cm.registerTaskManager(probe.getRef(), ici, resources, 1, leaderSessionID); } catch (Exception e) { // good } @@ -182,12 +178,12 @@ public void testReportHeartbeat() { JavaTestKit probe2 = new JavaTestKit(system); JavaTestKit probe3 = new JavaTestKit(system); - InstanceID instanceID1 = cm.registerTaskManager(probe1.getRef(), resID1, - ici1, hardwareDescription, 1, leaderSessionID); - InstanceID instanceID2 = cm.registerTaskManager(probe2.getRef(), resID2, - ici2, hardwareDescription, 1, leaderSessionID); - InstanceID instanceID3 = cm.registerTaskManager(probe3.getRef(), resID3, - ici3, hardwareDescription, 1, leaderSessionID); + InstanceID instanceID1 = cm.registerTaskManager( + probe1.getRef(), ici1, hardwareDescription, 1, leaderSessionID); + InstanceID instanceID2 = cm.registerTaskManager( + probe2.getRef(), ici2, hardwareDescription, 1, leaderSessionID); + InstanceID instanceID3 = cm.registerTaskManager( + probe3.getRef(), ici3, hardwareDescription, 1, leaderSessionID); // report some immediate heart beats assertTrue(cm.reportHeartBeat(instanceID1, new byte[] {})); @@ -241,8 +237,7 @@ public void testShutdown() { TaskManagerLocation ici = new TaskManagerLocation(resID, address, 20000); JavaTestKit probe = new JavaTestKit(system); - cm.registerTaskManager(probe.getRef(), resID, - ici, resources, 1, leaderSessionID); + cm.registerTaskManager(probe.getRef(), ici, resources, 1, leaderSessionID); fail("Should raise exception in shutdown state"); } catch (IllegalStateException e) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceTest.java index 82d3723b92d98..aee62fd3e7797 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceTest.java @@ -42,7 +42,7 @@ public void testAllocatingAndCancellingSlots() { TaskManagerLocation connection = new TaskManagerLocation(resourceID, address, 10001); Instance instance = new Instance(DummyActorGateway.INSTANCE, connection, - resourceID, new InstanceID(), hardwareDescription, 4); + new InstanceID(), hardwareDescription, 4); assertEquals(4, instance.getTotalNumberOfSlots()); assertEquals(4, instance.getNumberOfAvailableSlots()); @@ -105,7 +105,7 @@ public void testInstanceDies() { TaskManagerLocation connection = new TaskManagerLocation(resourceID, address, 10001); Instance instance = new Instance(DummyActorGateway.INSTANCE, connection, - resourceID, new InstanceID(), hardwareDescription, 3); + new InstanceID(), hardwareDescription, 3); assertEquals(3, instance.getNumberOfAvailableSlots()); @@ -137,7 +137,7 @@ public void testCancelAllSlots() { TaskManagerLocation connection = new TaskManagerLocation(resourceID, address, 10001); Instance instance = new Instance(DummyActorGateway.INSTANCE, connection, - resourceID, new InstanceID(), hardwareDescription, 3); + new InstanceID(), hardwareDescription, 3); assertEquals(3, instance.getNumberOfAvailableSlots()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SharedSlotsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SharedSlotsTest.java index 2c40e89325dd4..0edef5e2dc051 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SharedSlotsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SharedSlotsTest.java @@ -132,7 +132,7 @@ public void allocateSimpleSlotsAndReleaseFromRoot() { assertEquals(Locality.LOCAL, sub1.getLocality()); assertEquals(1, sub1.getNumberLeaves()); assertEquals(vid1, sub1.getGroupID()); - assertEquals(instance.getResourceId(), sub1.getTaskManagerID()); + assertEquals(instance.getTaskManagerID(), sub1.getTaskManagerID()); assertEquals(jobId, sub1.getJobID()); assertEquals(sharedSlot, sub1.getParent()); assertEquals(sharedSlot, sub1.getRoot()); @@ -151,7 +151,7 @@ public void allocateSimpleSlotsAndReleaseFromRoot() { assertEquals(Locality.UNCONSTRAINED, sub2.getLocality()); assertEquals(1, sub2.getNumberLeaves()); assertEquals(vid2, sub2.getGroupID()); - assertEquals(instance.getResourceId(), sub2.getTaskManagerID()); + assertEquals(instance.getTaskManagerID(), sub2.getTaskManagerID()); assertEquals(jobId, sub2.getJobID()); assertEquals(sharedSlot, sub2.getParent()); assertEquals(sharedSlot, sub2.getRoot()); @@ -163,14 +163,14 @@ public void allocateSimpleSlotsAndReleaseFromRoot() { assertEquals(1, assignment.getNumberOfAvailableSlotsForGroup(vid3)); assertEquals(1, assignment.getNumberOfAvailableSlotsForGroup(vid4)); - SimpleSlot sub3 = assignment.getSlotForTask(vid3, Collections.singleton(instance.getInstanceConnectionInfo())); + SimpleSlot sub3 = assignment.getSlotForTask(vid3, Collections.singleton(instance.getTaskManagerLocation())); assertNotNull(sub3); assertNull(sub3.getExecutedVertex()); assertEquals(Locality.LOCAL, sub3.getLocality()); assertEquals(1, sub3.getNumberLeaves()); assertEquals(vid3, sub3.getGroupID()); - assertEquals(instance.getResourceId(), sub3.getTaskManagerID()); + assertEquals(instance.getTaskManagerID(), sub3.getTaskManagerID()); assertEquals(jobId, sub3.getJobID()); assertEquals(sharedSlot, sub3.getParent()); assertEquals(sharedSlot, sub3.getRoot()); @@ -183,14 +183,14 @@ public void allocateSimpleSlotsAndReleaseFromRoot() { assertEquals(1, assignment.getNumberOfAvailableSlotsForGroup(vid4)); SimpleSlot sub4 = assignment.getSlotForTask(vid4, - Collections.singleton(SchedulerTestUtils.getRandomInstance(1).getInstanceConnectionInfo())); + Collections.singleton(SchedulerTestUtils.getRandomInstance(1).getTaskManagerLocation())); assertNotNull(sub4); assertNull(sub4.getExecutedVertex()); assertEquals(Locality.NON_LOCAL, sub4.getLocality()); assertEquals(1, sub4.getNumberLeaves()); assertEquals(vid4, sub4.getGroupID()); - assertEquals(instance.getResourceId(), sub4.getTaskManagerID()); + assertEquals(instance.getTaskManagerID(), sub4.getTaskManagerID()); assertEquals(jobId, sub4.getJobID()); assertEquals(sharedSlot, sub4.getParent()); assertEquals(sharedSlot, sub4.getRoot()); @@ -456,7 +456,7 @@ public void testAllocateAndReleaseTwoLevels() { assertNotNull(constraint.getSharedSlot()); assertTrue(constraint.isAssigned()); assertTrue(constraint.isAssignedAndAlive()); - assertEquals(instance.getInstanceConnectionInfo(), constraint.getLocation()); + assertEquals(instance.getTaskManagerLocation(), constraint.getLocation()); SimpleSlot tailSlot = assignment.getSlotForTask(constraint, Collections.emptySet()); @@ -475,7 +475,7 @@ public void testAllocateAndReleaseTwoLevels() { assertTrue(tailSlot.isReleased()); assertTrue(constraint.isAssigned()); assertFalse(constraint.isAssignedAndAlive()); - assertEquals(instance.getInstanceConnectionInfo(), constraint.getLocation()); + assertEquals(instance.getTaskManagerLocation(), constraint.getLocation()); // we should have resources again for the co-location constraint assertEquals(1, assignment.getNumberOfAvailableSlotsForGroup(constraint.getGroupId())); @@ -488,10 +488,10 @@ public void testAllocateAndReleaseTwoLevels() { assertEquals(0, assignment.getNumberOfAvailableSlotsForGroup(constraint.getGroupId())); // verify some basic properties of the slots - assertEquals(instance.getResourceId(), sourceSlot.getTaskManagerID()); - assertEquals(instance.getResourceId(), headSlot.getTaskManagerID()); - assertEquals(instance.getResourceId(), tailSlot.getTaskManagerID()); - assertEquals(instance.getResourceId(), sinkSlot.getTaskManagerID()); + assertEquals(instance.getTaskManagerID(), sourceSlot.getTaskManagerID()); + assertEquals(instance.getTaskManagerID(), headSlot.getTaskManagerID()); + assertEquals(instance.getTaskManagerID(), tailSlot.getTaskManagerID()); + assertEquals(instance.getTaskManagerID(), sinkSlot.getTaskManagerID()); assertEquals(sourceId, sourceSlot.getGroupID()); assertEquals(sinkId, sinkSlot.getGroupID()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotTest.java index 82c2a740cd620..c690d36b70c6f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotTest.java @@ -150,7 +150,7 @@ public static SimpleSlot getSlot() throws Exception { TaskManagerLocation connection = new TaskManagerLocation(resourceID, address, 10001); Instance instance = new Instance(DummyActorGateway.INSTANCE, connection, - resourceID, new InstanceID(), hardwareDescription, 1); + new InstanceID(), hardwareDescription, 1); return instance.allocateSimpleSlot(new JobID()); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewTest.java index fff7bc69a6f70..5722cac11a6f0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewTest.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.io.network.partition; -import com.google.common.collect.Lists; import org.apache.flink.runtime.io.disk.iomanager.BufferFileWriter; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode; @@ -36,6 +35,7 @@ import org.junit.runners.Parameterized; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.List; @@ -121,7 +121,7 @@ public void testReadMultipleFilesWithSingleBufferPool() throws Exception { } } - final List> results = Lists.newArrayList(); + final List> results = new ArrayList<>(); // Submit the consuming tasks for (ResultSubpartitionView view : readers) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraintTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraintTest.java index 3bd4368ec79c3..1344aef289054 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraintTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraintTest.java @@ -140,7 +140,7 @@ public void testAssignSlotAndLockLocation() { // now, the location is assigned and we have a location assertTrue(constraint.isAssigned()); assertTrue(constraint.isAssignedAndAlive()); - assertEquals(instance2, constraint.getLocation()); + assertEquals(instance2.getTaskManagerLocation(), constraint.getLocation()); // release the slot slot2_1.releaseSlot(); @@ -148,7 +148,7 @@ public void testAssignSlotAndLockLocation() { // we should still have a location assertTrue(constraint.isAssigned()); assertFalse(constraint.isAssignedAndAlive()); - assertEquals(instance2, constraint.getLocation()); + assertEquals(instance2.getTaskManagerLocation(), constraint.getLocation()); // we can not assign a different location try { @@ -167,7 +167,7 @@ public void testAssignSlotAndLockLocation() { assertTrue(constraint.isAssigned()); assertTrue(constraint.isAssignedAndAlive()); - assertEquals(instance2, constraint.getLocation()); + assertEquals(instance2.getTaskManagerLocation(), constraint.getLocation()); } catch (Exception e) { e.printStackTrace(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java index 5b7d18a3c20ea..eab4fea69a372 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java @@ -326,8 +326,8 @@ public void testGetsNonLocalFromSharingGroupFirst() { Instance i1 = getRandomInstance(1); Instance i2 = getRandomInstance(1); - TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); - TaskManagerLocation loc2 = i2.getInstanceConnectionInfo(); + TaskManagerLocation loc1 = i1.getTaskManagerLocation(); + TaskManagerLocation loc2 = i2.getTaskManagerLocation(); scheduler.newInstanceAvailable(i2); scheduler.newInstanceAvailable(i1); @@ -398,8 +398,8 @@ public void testSlotReleasedInBetween() { Instance i1 = getRandomInstance(1); Instance i2 = getRandomInstance(1); - TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); - TaskManagerLocation loc2 = i2.getInstanceConnectionInfo(); + TaskManagerLocation loc1 = i1.getTaskManagerLocation(); + TaskManagerLocation loc2 = i2.getTaskManagerLocation(); scheduler.newInstanceAvailable(i2); scheduler.newInstanceAvailable(i1); @@ -425,8 +425,8 @@ public void testSlotReleasedInBetween() { SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup, cc2)); // still preserves the previous instance mapping) - assertEquals(i1.getResourceId(), s3.getTaskManagerID()); - assertEquals(i2.getResourceId(), s4.getTaskManagerID()); + assertEquals(i1.getTaskManagerID(), s3.getTaskManagerID()); + assertEquals(i2.getTaskManagerID(), s4.getTaskManagerID()); s3.releaseSlot(); s4.releaseSlot(); @@ -455,8 +455,8 @@ public void testSlotReleasedInBetweenAndNoNewLocal() { Instance i1 = getRandomInstance(1); Instance i2 = getRandomInstance(1); - TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); - TaskManagerLocation loc2 = i2.getInstanceConnectionInfo(); + TaskManagerLocation loc1 = i1.getTaskManagerLocation(); + TaskManagerLocation loc2 = i2.getTaskManagerLocation(); scheduler.newInstanceAvailable(i2); scheduler.newInstanceAvailable(i1); @@ -516,7 +516,7 @@ public void testScheduleOutOfOrder() { Instance i1 = getRandomInstance(1); Instance i2 = getRandomInstance(1); - TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); + TaskManagerLocation loc1 = i1.getTaskManagerLocation(); scheduler.newInstanceAvailable(i2); scheduler.newInstanceAvailable(i1); @@ -580,8 +580,8 @@ public void nonColocationFollowsCoLocation() { Instance i1 = getRandomInstance(1); Instance i2 = getRandomInstance(1); - TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); - TaskManagerLocation loc2 = i2.getInstanceConnectionInfo(); + TaskManagerLocation loc1 = i1.getTaskManagerLocation(); + TaskManagerLocation loc2 = i2.getTaskManagerLocation(); scheduler.newInstanceAvailable(i2); scheduler.newInstanceAvailable(i1); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java index a683834553d32..fd0523b985932 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java @@ -108,10 +108,10 @@ public void scheduleSingleVertexType() { // make sure we have two slots on the first instance, and two on the second int c = 0; - c += (s5.getTaskManagerID().equals(i1.getResourceId())) ? 1 : -1; - c += (s6.getTaskManagerID().equals(i1.getResourceId())) ? 1 : -1; - c += (s7.getTaskManagerID().equals(i1.getResourceId())) ? 1 : -1; - c += (s8.getTaskManagerID().equals(i1.getResourceId())) ? 1 : -1; + c += (s5.getTaskManagerID().equals(i1.getTaskManagerID())) ? 1 : -1; + c += (s6.getTaskManagerID().equals(i1.getTaskManagerID())) ? 1 : -1; + c += (s7.getTaskManagerID().equals(i1.getTaskManagerID())) ? 1 : -1; + c += (s8.getTaskManagerID().equals(i1.getTaskManagerID())) ? 1 : -1; assertEquals(0, c); // release all @@ -637,8 +637,8 @@ public void testLocalizedAssignment1() { Instance i1 = getRandomInstance(2); Instance i2 = getRandomInstance(2); - TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); - TaskManagerLocation loc2 = i2.getInstanceConnectionInfo(); + TaskManagerLocation loc1 = i1.getTaskManagerLocation(); + TaskManagerLocation loc2 = i2.getTaskManagerLocation(); Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext()); scheduler.newInstanceAvailable(i1); @@ -690,8 +690,8 @@ public void testLocalizedAssignment2() { Instance i1 = getRandomInstance(2); Instance i2 = getRandomInstance(2); - TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); - TaskManagerLocation loc2 = i2.getInstanceConnectionInfo(); + TaskManagerLocation loc1 = i1.getTaskManagerLocation(); + TaskManagerLocation loc2 = i2.getTaskManagerLocation(); Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext()); scheduler.newInstanceAvailable(i1); @@ -743,7 +743,7 @@ public void testLocalizedAssignment3() { Instance i1 = getRandomInstance(2); Instance i2 = getRandomInstance(2); - TaskManagerLocation loc1 = i1.getInstanceConnectionInfo(); + TaskManagerLocation loc1 = i1.getTaskManagerLocation(); Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext()); scheduler.newInstanceAvailable(i1); @@ -771,12 +771,12 @@ public void testLocalizedAssignment3() { assertEquals(0, i1.getNumberOfAvailableSlots()); assertEquals(0, i2.getNumberOfAvailableSlots()); - assertEquals(i1.getResourceId(), s1.getTaskManagerID()); - assertEquals(i1.getResourceId(), s2.getTaskManagerID()); - assertEquals(i1.getResourceId(), s3.getTaskManagerID()); - assertEquals(i1.getResourceId(), s4.getTaskManagerID()); - assertEquals(i2.getResourceId(), s5.getTaskManagerID()); - assertEquals(i2.getResourceId(), s6.getTaskManagerID()); + assertEquals(i1.getTaskManagerID(), s1.getTaskManagerID()); + assertEquals(i1.getTaskManagerID(), s2.getTaskManagerID()); + assertEquals(i1.getTaskManagerID(), s3.getTaskManagerID()); + assertEquals(i1.getTaskManagerID(), s4.getTaskManagerID()); + assertEquals(i2.getTaskManagerID(), s5.getTaskManagerID()); + assertEquals(i2.getTaskManagerID(), s6.getTaskManagerID()); // check the scheduler's bookkeeping assertEquals(4, scheduler.getNumberOfLocalizedAssignments()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java index eef27a8d73a8f..d040ec40fd9ad 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java @@ -69,8 +69,7 @@ public static Instance getRandomInstance(int numSlots) { final long GB = 1024L*1024*1024; HardwareDescription resources = new HardwareDescription(4, 4*GB, 3*GB, 2*GB); - return new Instance(DummyActorGateway.INSTANCE, ci, resourceID, - new InstanceID(), resources, numSlots); + return new Instance(DummyActorGateway.INSTANCE, ci, new InstanceID(), resources, numSlots); } @@ -88,7 +87,7 @@ public static Execution getDummyTask() { public static Execution getTestVertex(Instance... preferredInstances) { List locations = new ArrayList<>(preferredInstances.length); for (Instance i : preferredInstances) { - locations.add(i.getInstanceConnectionInfo()); + locations.add(i.getTaskManagerLocation()); } return getTestVertex(locations); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureTest.java index d9c100c080ed3..ea0d2cc25e243 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureTest.java @@ -56,9 +56,9 @@ public void slotAllocated(SimpleSlot slot) {} final Instance instance2 = SchedulerTestUtils.getRandomInstance(1); final SimpleSlot slot1 = new SimpleSlot(new JobID(), instance1, - instance1.getInstanceConnectionInfo(), 0, instance1.getActorGateway(), null, null); + instance1.getTaskManagerLocation(), 0, instance1.getActorGateway(), null, null); final SimpleSlot slot2 = new SimpleSlot(new JobID(), instance2, - instance2.getInstanceConnectionInfo(), 0, instance2.getActorGateway(), null, null); + instance2.getTaskManagerLocation(), 0, instance2.getActorGateway(), null, null); future.setSlot(slot1); try { @@ -85,7 +85,7 @@ public void setWithAction() { final Instance instance = SchedulerTestUtils.getRandomInstance(1); final SimpleSlot thisSlot = new SimpleSlot(new JobID(), instance, - instance.getInstanceConnectionInfo(), 0, instance.getActorGateway(), null, null); + instance.getTaskManagerLocation(), 0, instance.getActorGateway(), null, null); SlotAllocationFuture future = new SlotAllocationFuture(); @@ -108,7 +108,7 @@ public void slotAllocated(SimpleSlot slot) { final Instance instance = SchedulerTestUtils.getRandomInstance(1); final SimpleSlot thisSlot = new SimpleSlot(new JobID(), instance, - instance.getInstanceConnectionInfo(), 0, instance.getActorGateway(), null, null); + instance.getTaskManagerLocation(), 0, instance.getActorGateway(), null, null); SlotAllocationFuture future = new SlotAllocationFuture(); future.setSlot(thisSlot); @@ -141,7 +141,7 @@ public void setSync() { final Instance instance = SchedulerTestUtils.getRandomInstance(1); final SimpleSlot thisSlot = new SimpleSlot(new JobID(), instance, - instance.getInstanceConnectionInfo(), 0, instance.getActorGateway(), null, null); + instance.getTaskManagerLocation(), 0, instance.getActorGateway(), null, null); final SlotAllocationFuture future = new SlotAllocationFuture(); @@ -181,7 +181,7 @@ public void run() { final Instance instance = SchedulerTestUtils.getRandomInstance(1); final SimpleSlot thisSlot = new SimpleSlot(new JobID(), instance, - instance.getInstanceConnectionInfo(), 0, instance.getActorGateway(), null, null); + instance.getTaskManagerLocation(), 0, instance.getActorGateway(), null, null); final SlotAllocationFuture future = new SlotAllocationFuture(); future.setSlot(thisSlot); From a2c937a95421716508d8ddcc103351c6cb0f0c41 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 31 Aug 2016 13:52:45 +0200 Subject: [PATCH 019/299] [FLINK-4525] [core] (followup) Remove remaining redundant code for pre-defined strictly local assignments. --- .../executiongraph/ExecutionVertex.java | 13 - .../VertexLocationConstraintTest.java | 456 ------------------ 2 files changed, 469 deletions(-) delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java index f02647e001ade..88e1b889c7ea6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java @@ -97,8 +97,6 @@ public class ExecutionVertex { private volatile Execution currentExecution; // this field must never be null - private volatile List locationConstraintInstances; - private volatile boolean scheduleLocalOnly; // -------------------------------------------------------------------------------------------- @@ -351,10 +349,6 @@ else if (numSources < parallelism) { } } - public void setLocationConstraintHosts(List instances) { - this.locationConstraintInstances = instances; - } - public void setScheduleLocalOnly(boolean scheduleLocalOnly) { if (scheduleLocalOnly && inputEdges != null && inputEdges.length > 0) { throw new IllegalArgumentException("Strictly local scheduling is only supported for sources."); @@ -376,12 +370,6 @@ public boolean isScheduleLocalOnly() { * @return The preferred locations for this vertex execution, or null, if there is no preference. */ public Iterable getPreferredLocations() { - // if we have hard location constraints, use those - List constraintInstances = this.locationConstraintInstances; - if (constraintInstances != null && !constraintInstances.isEmpty()) { - return constraintInstances; - } - // otherwise, base the preferred locations on the input connections if (inputEdges == null) { return Collections.emptySet(); @@ -570,7 +558,6 @@ public void prepareForArchiving() throws IllegalStateException { this.resultPartitions = null; this.inputEdges = null; this.locationConstraint = null; - this.locationConstraintInstances = null; } public void cachePartitionInfo(PartialInputChannelDeploymentDescriptor partitionInfo){ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java deleted file mode 100644 index a1f3345541b53..0000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexLocationConstraintTest.java +++ /dev/null @@ -1,456 +0,0 @@ -/* - * 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.runtime.executiongraph; - -import static org.junit.Assert.*; -import static org.mockito.Mockito.*; - -import java.net.InetAddress; -import java.util.Arrays; -import java.util.Collections; -import java.util.concurrent.TimeUnit; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; -import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.instance.DummyActorGateway; -import org.apache.flink.runtime.instance.HardwareDescription; -import org.apache.flink.runtime.instance.Instance; -import org.apache.flink.runtime.taskmanager.TaskManagerLocation; -import org.apache.flink.runtime.instance.InstanceID; -import org.apache.flink.runtime.instance.SimpleSlot; -import org.apache.flink.runtime.jobgraph.JobVertex; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException; -import org.apache.flink.runtime.jobmanager.scheduler.Scheduler; -import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; -import org.apache.flink.runtime.operators.testutils.DummyInvokable; -import org.apache.flink.runtime.testingUtils.TestingUtils; -import org.apache.flink.util.SerializedValue; -import org.junit.Test; - -import scala.concurrent.duration.FiniteDuration; - -public class VertexLocationConstraintTest { - - private static final FiniteDuration timeout = new FiniteDuration(100, TimeUnit.SECONDS); - - @Test - public void testScheduleWithConstraint1() { - try { - final byte[] address1 = { 10, 0, 1, 4 }; - final byte[] address2 = { 10, 0, 1, 5 }; - final byte[] address3 = { 10, 0, 1, 6 }; - - final String hostname1 = "host1"; - final String hostname2 = "host2"; - final String hostname3 = "host3"; - - // prepare the scheduler - Instance instance1 = getInstance(address1, 6789, hostname1); - Instance instance2 = getInstance(address2, 6789, hostname2); - Instance instance3 = getInstance(address3, 6789, hostname3); - - Scheduler scheduler = new Scheduler(TestingUtils.defaultExecutionContext()); - scheduler.newInstanceAvailable(instance1); - scheduler.newInstanceAvailable(instance2); - scheduler.newInstanceAvailable(instance3); - - // prepare the execution graph - JobVertex jobVertex = new JobVertex("test vertex", new JobVertexID()); - jobVertex.setInvokableClass(DummyInvokable.class); - jobVertex.setParallelism(2); - JobGraph jg = new JobGraph("test job", jobVertex); - - ExecutionGraph eg = new ExecutionGraph( - TestingUtils.defaultExecutionContext(), - jg.getJobID(), - jg.getName(), - jg.getJobConfiguration(), - new SerializedValue<>(new ExecutionConfig()), - timeout, - new NoRestartStrategy()); - eg.attachJobGraph(Collections.singletonList(jobVertex)); - - ExecutionJobVertex ejv = eg.getAllVertices().get(jobVertex.getID()); - ExecutionVertex[] vertices = ejv.getTaskVertices(); - - vertices[0].setLocationConstraintHosts(Arrays.asList(instance1.getInstanceConnectionInfo(), instance2.getInstanceConnectionInfo())); - vertices[1].setLocationConstraintHosts(Collections.singletonList(instance3.getInstanceConnectionInfo())); - - vertices[0].setScheduleLocalOnly(true); - vertices[1].setScheduleLocalOnly(true); - - ejv.scheduleAll(scheduler, false); - - SimpleSlot slot1 = vertices[0].getCurrentAssignedResource(); - SimpleSlot slot2 = vertices[1].getCurrentAssignedResource(); - - assertNotNull(slot1); - assertNotNull(slot2); - - ResourceID target1 = slot1.getTaskManagerID(); - ResourceID target2 = slot2.getTaskManagerID(); - - assertNotNull(target1); - assertNotNull(target2); - - assertTrue(target1 == instance1.getResourceId() || target1 == instance2.getResourceId()); - assertEquals(target2, instance3.getResourceId()); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testScheduleWithConstraint2() { - - // same test as above, which swapped host names to guard against "accidentally worked" because of - // the order in which requests are handles by internal data structures - - try { - final byte[] address1 = { 10, 0, 1, 4 }; - final byte[] address2 = { 10, 0, 1, 5 }; - final byte[] address3 = { 10, 0, 1, 6 }; - - final String hostname1 = "host1"; - final String hostname2 = "host2"; - final String hostname3 = "host3"; - - // prepare the scheduler - Instance instance1 = getInstance(address1, 6789, hostname1); - Instance instance2 = getInstance(address2, 6789, hostname2); - Instance instance3 = getInstance(address3, 6789, hostname3); - - Scheduler scheduler = new Scheduler(TestingUtils.defaultExecutionContext()); - scheduler.newInstanceAvailable(instance1); - scheduler.newInstanceAvailable(instance2); - scheduler.newInstanceAvailable(instance3); - - // prepare the execution graph - JobVertex jobVertex = new JobVertex("test vertex", new JobVertexID()); - jobVertex.setInvokableClass(DummyInvokable.class); - jobVertex.setParallelism(2); - JobGraph jg = new JobGraph("test job", jobVertex); - - ExecutionGraph eg = new ExecutionGraph( - TestingUtils.defaultExecutionContext(), - jg.getJobID(), - jg.getName(), - jg.getJobConfiguration(), - new SerializedValue<>(new ExecutionConfig()), - timeout, - new NoRestartStrategy()); - eg.attachJobGraph(Collections.singletonList(jobVertex)); - - ExecutionJobVertex ejv = eg.getAllVertices().get(jobVertex.getID()); - ExecutionVertex[] vertices = ejv.getTaskVertices(); - - vertices[0].setLocationConstraintHosts(Collections.singletonList(instance3.getInstanceConnectionInfo())); - vertices[1].setLocationConstraintHosts(Arrays.asList(instance1.getInstanceConnectionInfo(), instance2.getInstanceConnectionInfo())); - - vertices[0].setScheduleLocalOnly(true); - vertices[1].setScheduleLocalOnly(true); - - ejv.scheduleAll(scheduler, false); - - SimpleSlot slot1 = vertices[0].getCurrentAssignedResource(); - SimpleSlot slot2 = vertices[1].getCurrentAssignedResource(); - - assertNotNull(slot1); - assertNotNull(slot2); - - ResourceID target1 = slot1.getTaskManagerID(); - ResourceID target2 = slot2.getTaskManagerID(); - - assertTrue(target1 == instance3.getResourceId()); - assertTrue(target2 == instance1.getResourceId() || target2 == instance2.getResourceId()); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testScheduleWithConstraintAndSlotSharing() { - try { - final byte[] address1 = { 10, 0, 1, 4 }; - final byte[] address2 = { 10, 0, 1, 5 }; - final byte[] address3 = { 10, 0, 1, 6 }; - - final String hostname1 = "host1"; - final String hostname2 = "host2"; - final String hostname3 = "host3"; - - // prepare the scheduler - Instance instance1 = getInstance(address1, 6789, hostname1); - Instance instance2 = getInstance(address2, 6789, hostname2); - Instance instance3 = getInstance(address3, 6789, hostname3); - - Scheduler scheduler = new Scheduler(TestingUtils.defaultExecutionContext()); - scheduler.newInstanceAvailable(instance1); - scheduler.newInstanceAvailable(instance2); - scheduler.newInstanceAvailable(instance3); - - // prepare the execution graph - JobVertex jobVertex1 = new JobVertex("v1", new JobVertexID()); - JobVertex jobVertex2 = new JobVertex("v2", new JobVertexID()); - jobVertex1.setInvokableClass(DummyInvokable.class); - jobVertex2.setInvokableClass(DummyInvokable.class); - jobVertex1.setParallelism(2); - jobVertex2.setParallelism(3); - - SlotSharingGroup sharingGroup = new SlotSharingGroup(); - jobVertex1.setSlotSharingGroup(sharingGroup); - jobVertex2.setSlotSharingGroup(sharingGroup); - - JobGraph jg = new JobGraph("test job", jobVertex1, jobVertex2); - - ExecutionGraph eg = new ExecutionGraph( - TestingUtils.defaultExecutionContext(), - jg.getJobID(), - jg.getName(), - jg.getJobConfiguration(), - new SerializedValue<>(new ExecutionConfig()), - timeout, - new NoRestartStrategy()); - eg.attachJobGraph(Arrays.asList(jobVertex1, jobVertex2)); - - ExecutionJobVertex ejv = eg.getAllVertices().get(jobVertex1.getID()); - ExecutionVertex[] vertices = ejv.getTaskVertices(); - - vertices[0].setLocationConstraintHosts(Arrays.asList(instance1.getInstanceConnectionInfo(), instance2.getInstanceConnectionInfo())); - vertices[1].setLocationConstraintHosts(Collections.singletonList(instance3.getInstanceConnectionInfo())); - - vertices[0].setScheduleLocalOnly(true); - vertices[1].setScheduleLocalOnly(true); - - ejv.scheduleAll(scheduler, false); - - SimpleSlot slot1 = vertices[0].getCurrentAssignedResource(); - SimpleSlot slot2 = vertices[1].getCurrentAssignedResource(); - - assertNotNull(slot1); - assertNotNull(slot2); - - ResourceID target1 = slot1.getTaskManagerID(); - ResourceID target2 = slot2.getTaskManagerID(); - - assertTrue(target1 == instance1.getResourceId() || target1 == instance2.getResourceId()); - assertTrue(target2 == instance3.getResourceId()); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testScheduleWithUnfulfillableConstraint() { - - // same test as above, which swapped host names to guard against "accidentally worked" because of - // the order in which requests are handles by internal data structures - - try { - final byte[] address1 = { 10, 0, 1, 4 }; - final byte[] address2 = { 10, 0, 1, 5 }; - - final String hostname1 = "host1"; - final String hostname2 = "host2"; - - // prepare the scheduler - Instance instance1 = getInstance(address1, 6789, hostname1); - Instance instance2 = getInstance(address2, 6789, hostname2); - - Scheduler scheduler = new Scheduler(TestingUtils.defaultExecutionContext()); - scheduler.newInstanceAvailable(instance1); - - // prepare the execution graph - JobVertex jobVertex = new JobVertex("test vertex", new JobVertexID()); - jobVertex.setInvokableClass(DummyInvokable.class); - jobVertex.setParallelism(1); - JobGraph jg = new JobGraph("test job", jobVertex); - - ExecutionGraph eg = new ExecutionGraph( - TestingUtils.defaultExecutionContext(), - jg.getJobID(), - jg.getName(), - jg.getJobConfiguration(), - new SerializedValue<>(new ExecutionConfig()), - timeout, - new NoRestartStrategy()); - eg.attachJobGraph(Collections.singletonList(jobVertex)); - - ExecutionJobVertex ejv = eg.getAllVertices().get(jobVertex.getID()); - ExecutionVertex[] vertices = ejv.getTaskVertices(); - - vertices[0].setLocationConstraintHosts(Collections.singletonList(instance2.getInstanceConnectionInfo())); - vertices[0].setScheduleLocalOnly(true); - - try { - ejv.scheduleAll(scheduler, false); - fail("This should fail with a NoResourceAvailableException"); - } - catch (NoResourceAvailableException e) { - // bam! we are good... - assertTrue(e.getMessage().contains(hostname2)); - } - catch (Exception e) { - fail("Wrong exception type"); - } - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testScheduleWithUnfulfillableConstraintInSharingGroup() { - - // same test as above, which swapped host names to guard against "accidentally worked" because of - // the order in which requests are handles by internal data structures - - try { - final byte[] address1 = { 10, 0, 1, 4 }; - final byte[] address2 = { 10, 0, 1, 5 }; - - final String hostname1 = "host1"; - final String hostname2 = "host2"; - - // prepare the scheduler - Instance instance1 = getInstance(address1, 6789, hostname1); - Instance instance2 = getInstance(address2, 6789, hostname2); - - Scheduler scheduler = new Scheduler(TestingUtils.defaultExecutionContext()); - scheduler.newInstanceAvailable(instance1); - - // prepare the execution graph - JobVertex jobVertex1 = new JobVertex("v1", new JobVertexID()); - JobVertex jobVertex2 = new JobVertex("v2", new JobVertexID()); - - jobVertex1.setInvokableClass(DummyInvokable.class); - jobVertex2.setInvokableClass(DummyInvokable.class); - - jobVertex1.setParallelism(1); - jobVertex2.setParallelism(1); - - JobGraph jg = new JobGraph("test job", jobVertex1, jobVertex2); - - SlotSharingGroup sharingGroup = new SlotSharingGroup(); - jobVertex1.setSlotSharingGroup(sharingGroup); - jobVertex2.setSlotSharingGroup(sharingGroup); - - ExecutionGraph eg = new ExecutionGraph( - TestingUtils.defaultExecutionContext(), - jg.getJobID(), - jg.getName(), - jg.getJobConfiguration(), - new SerializedValue<>(new ExecutionConfig()), - timeout, - new NoRestartStrategy()); - eg.attachJobGraph(Arrays.asList(jobVertex1, jobVertex2)); - - ExecutionJobVertex ejv = eg.getAllVertices().get(jobVertex1.getID()); - ExecutionVertex[] vertices = ejv.getTaskVertices(); - - vertices[0].setLocationConstraintHosts(Collections.singletonList(instance2.getInstanceConnectionInfo())); - vertices[0].setScheduleLocalOnly(true); - - try { - ejv.scheduleAll(scheduler, false); - fail("This should fail with a NoResourceAvailableException"); - } - catch (NoResourceAvailableException e) { - // bam! we are good... - assertTrue(e.getMessage().contains(hostname2)); - } - catch (Exception e) { - fail("Wrong exception type"); - } - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testArchivingClearsFields() { - try { - JobVertex vertex = new JobVertex("test vertex", new JobVertexID()); - JobGraph jg = new JobGraph("test job", vertex); - - ExecutionGraph eg = new ExecutionGraph( - TestingUtils.defaultExecutionContext(), - jg.getJobID(), - jg.getName(), - jg.getJobConfiguration(), - new SerializedValue<>(new ExecutionConfig()), - timeout, - new NoRestartStrategy()); - eg.attachJobGraph(Collections.singletonList(vertex)); - - ExecutionVertex ev = eg.getAllVertices().get(vertex.getID()).getTaskVertices()[0]; - - Instance instance = ExecutionGraphTestUtils.getInstance(DummyActorGateway.INSTANCE); - ev.setLocationConstraintHosts(Collections.singletonList(instance.getInstanceConnectionInfo())); - - assertNotNull(ev.getPreferredLocations()); - assertEquals(instance, ev.getPreferredLocations().iterator().next()); - - // transition to a final state - eg.fail(new Exception()); - - eg.prepareForArchiving(); - - assertTrue(ev.getPreferredLocations() == null || !ev.getPreferredLocations().iterator().hasNext()); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - // -------------------------------------------------------------------------------------------- - - public static Instance getInstance(byte[] ipAddress, int dataPort, String hostname) throws Exception { - HardwareDescription hardwareDescription = new HardwareDescription(4, 2L*1024*1024*1024, 1024*1024*1024, 512*1024*1024); - - TaskManagerLocation connection = mock(TaskManagerLocation.class); - when(connection.address()).thenReturn(InetAddress.getByAddress(ipAddress)); - when(connection.dataPort()).thenReturn(dataPort); - when(connection.addressString()).thenReturn(InetAddress.getByAddress(ipAddress).toString()); - when(connection.getHostname()).thenReturn(hostname); - when(connection.getFQDNHostname()).thenReturn(hostname); - - return new Instance( - new ExecutionGraphTestUtils.SimpleActorGateway( - TestingUtils.defaultExecutionContext()), - connection, - ResourceID.generate(), - new InstanceID(), - hardwareDescription, - 1); - } -} From 0b722144b40c6163568feab1a563702537c0d251 Mon Sep 17 00:00:00 2001 From: twalthr Date: Tue, 30 Aug 2016 17:21:13 +0200 Subject: [PATCH 020/299] [FLINK-3580] [table] Add current time point functions This closes #2441. --- docs/dev/table_api.md | 169 +++++++++++++++++- .../flink/api/scala/table/expressionDsl.scala | 72 ++++++++ .../api/table/codegen/CodeGenerator.scala | 102 +++++++++++ .../calls/CurrentTimePointCallGen.scala | 58 ++++++ .../table/codegen/calls/ScalarFunctions.scala | 25 +++ .../flink/api/table/expressions/time.scala | 47 +++++ .../api/table/validate/FunctionCatalog.scala | 8 +- .../expressions/NonDeterministicTests.scala | 89 +++++++++ .../expressions/ScalarFunctionsTest.scala | 45 +++++ 9 files changed, 613 insertions(+), 2 deletions(-) create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/CurrentTimePointCallGen.scala create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/NonDeterministicTests.scala diff --git a/docs/dev/table_api.md b/docs/dev/table_api.md index 68a2b956858af..9272ea3808de6 100644 --- a/docs/dev/table_api.md +++ b/docs/dev/table_api.md @@ -981,6 +981,8 @@ functionCall = composite , "." , functionIdentifier , [ "(" , [ expression , { " atom = ( "(" , expression , ")" ) | literal | nullLiteral | fieldReference ; +fieldReference = "*" | identifier ; + nullLiteral = "Null(" , dataType , ")" ; timeIntervalUnit = "YEAR" | "YEAR_TO_MONTH" | "MONTH" | "DAY" | "DAY_TO_HOUR" | "DAY_TO_MINUTE" | "DAY_TO_SECOND" | "HOUR" | "HOUR_TO_MINUTE" | "HOUR_TO_SECOND" | "MINUTE" | "MINUTE_TO_SECOND" | "SECOND" ; @@ -989,7 +991,7 @@ timePointUnit = "YEAR" | "MONTH" | "DAY" | "HOUR" | "MINUTE" | "SECOND" | "QUART {% endhighlight %} -Here, `literal` is a valid Java literal, `fieldReference` specifies a column in the data, and `functionIdentifier` specifies a supported scalar function. The +Here, `literal` is a valid Java literal, `fieldReference` specifies a column in the data (or all columns if `*` is used), and `functionIdentifier` specifies a supported scalar function. The column names and function names follow Java identifier syntax. Expressions specified as Strings can also use prefix notation instead of suffix notation to call operators and functions. If working with exact numeric values or large decimals is required, the Table API also supports Java's BigDecimal type. In the Scala Table API decimals can be defined by `BigDecimal("123456")` and in Java by appending a "p" for precise e.g. `123456p`. @@ -1521,6 +1523,61 @@ TIMEPOINT.ceil(TIMEINTERVALUNIT) + + + {% highlight java %} +currentDate() +{% endhighlight %} + + +

Returns the current SQL date in UTC time zone.

+ + + + + + {% highlight java %} +currentTime() +{% endhighlight %} + + +

Returns the current SQL time in UTC time zone.

+ + + + + + {% highlight java %} +currentTimestamp() +{% endhighlight %} + + +

Returns the current SQL timestamp in UTC time zone.

+ + + + + + {% highlight java %} +localTime() +{% endhighlight %} + + +

Returns the current SQL time in local time zone.

+ + + + + + {% highlight java %} +localTimestamp() +{% endhighlight %} + + +

Returns the current SQL timestamp in local time zone.

+ + + @@ -1828,6 +1885,61 @@ TIMEPOINT.ceil(TimeIntervalUnit) + + + {% highlight scala %} +currentDate() +{% endhighlight %} + + +

Returns the current SQL date in UTC time zone.

+ + + + + + {% highlight scala %} +currentTime() +{% endhighlight %} + + +

Returns the current SQL time in UTC time zone.

+ + + + + + {% highlight scala %} +currentTimestamp() +{% endhighlight %} + + +

Returns the current SQL timestamp in UTC time zone.

+ + + + + + {% highlight scala %} +localTime() +{% endhighlight %} + + +

Returns the current SQL time in local time zone.

+ + + + + + {% highlight scala %} +localTimestamp() +{% endhighlight %} + + +

Returns the current SQL timestamp in local time zone.

+ + + @@ -2093,6 +2205,61 @@ CEIL(TIMEPOINT TO TIMEINTERVALUNIT) + + + {% highlight sql %} +CURRENT_DATE +{% endhighlight %} + + +

Returns the current SQL date in UTC time zone.

+ + + + + + {% highlight sql %} +CURRENT_TIME +{% endhighlight %} + + +

Returns the current SQL time in UTC time zone.

+ + + + + + {% highlight sql %} +CURRENT_TIMESTAMP +{% endhighlight %} + + +

Returns the current SQL timestamp in UTC time zone.

+ + + + + + {% highlight sql %} +LOCALTIME +{% endhighlight %} + + +

Returns the current SQL time in local time zone.

+ + + + + + {% highlight sql %} +LOCALTIMESTAMP +{% endhighlight %} + + +

Returns the current SQL timestamp in local time zone.

+ + + diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/expressionDsl.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/expressionDsl.scala index 9bfe6c39d6bee..942b07e36f2eb 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/expressionDsl.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/expressionDsl.scala @@ -402,3 +402,75 @@ trait ImplicitExpressionConversions { implicit def sqlTime2Literal(sqlTime: Time): Expression = Literal(sqlTime) implicit def sqlTimestamp2Literal(sqlTimestamp: Timestamp): Expression = Literal(sqlTimestamp) } + +// ------------------------------------------------------------------------------------------------ +// Expressions with no parameters +// ------------------------------------------------------------------------------------------------ + +/** + * Returns the current SQL date in UTC time zone. + */ +object currentDate { + + /** + * Returns the current SQL date in UTC time zone. + */ + def apply(): Expression = { + CurrentDate() + } +} + +/** + * Returns the current SQL time in UTC time zone. + */ +object currentTime { + + /** + * Returns the current SQL time in UTC time zone. + */ + def apply(): Expression = { + CurrentTime() + } +} + +/** + * Returns the current SQL timestamp in UTC time zone. + */ +object currentTimestamp { + + /** + * Returns the current SQL timestamp in UTC time zone. + */ + def apply(): Expression = { + CurrentTimestamp() + } +} + +/** + * Returns the current SQL time in local time zone. + */ +object localTime { + + /** + * Returns the current SQL time in local time zone. + */ + def apply(): Expression = { + LocalTime() + } +} + +/** + * Returns the current SQL timestamp in local time zone. + */ +object localTimestamp { + + /** + * Returns the current SQL timestamp in local time zone. + */ + def apply(): Expression = { + LocalTimestamp() + } +} + + + diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala index 6463ff99331bc..39ee26cc2e13b 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala @@ -20,6 +20,7 @@ package org.apache.flink.api.table.codegen import java.math.{BigDecimal => JBigDecimal} +import org.apache.calcite.avatica.util.DateTimeUtils import org.apache.calcite.rex._ import org.apache.calcite.sql.SqlOperator import org.apache.calcite.sql.`type`.SqlTypeName._ @@ -106,6 +107,10 @@ class CodeGenerator( // we use a LinkedHashSet to keep the insertion order private val reusableInitStatements = mutable.LinkedHashSet[String]() + // set of statements that will be added only once per record + // we use a LinkedHashSet to keep the insertion order + private val reusablePerRecordStatements = mutable.LinkedHashSet[String]() + // map of initial input unboxing expressions that will be added only once // (inputTerm, index) -> expr private val reusableInputUnboxingExprs = mutable.Map[(String, Int), GeneratedExpression]() @@ -125,6 +130,13 @@ class CodeGenerator( reusableInitStatements.mkString("", "\n", "\n") } + /** + * @return code block of statements that need to be placed in the SAM of the Function + */ + def reusePerRecordCode(): String = { + reusablePerRecordStatements.mkString("", "\n", "\n") + } + /** * @return code block of statements that unbox input variables to a primitive variable * and a corresponding null flag variable @@ -234,6 +246,7 @@ class CodeGenerator( @Override public ${samHeader._1} throws Exception { ${samHeader._2.mkString("\n")} + ${reusePerRecordCode()} ${reuseInputUnboxingCode()} $bodyCode } @@ -1243,4 +1256,93 @@ class CodeGenerator( reusableInitStatements.add(constructorAccessibility) fieldTerm } + + /** + * Adds a reusable timestamp to the beginning of the SAM of the generated [[Function]]. + */ + def addReusableTimestamp(): String = { + val fieldTerm = s"timestamp" + + val field = + s""" + |final long $fieldTerm = java.lang.System.currentTimeMillis(); + |""".stripMargin + reusablePerRecordStatements.add(field) + fieldTerm + } + + /** + * Adds a reusable local timestamp to the beginning of the SAM of the generated [[Function]]. + */ + def addReusableLocalTimestamp(): String = { + val fieldTerm = s"localtimestamp" + + val timestamp = addReusableTimestamp() + + val field = + s""" + |final long $fieldTerm = $timestamp + java.util.TimeZone.getDefault().getOffset(timestamp); + |""".stripMargin + reusablePerRecordStatements.add(field) + fieldTerm + } + + /** + * Adds a reusable time to the beginning of the SAM of the generated [[Function]]. + */ + def addReusableTime(): String = { + val fieldTerm = s"time" + + val timestamp = addReusableTimestamp() + + // adopted from org.apache.calcite.runtime.SqlFunctions.currentTime() + val field = + s""" + |final int $fieldTerm = (int) ($timestamp % ${DateTimeUtils.MILLIS_PER_DAY}); + |if (time < 0) { + | time += ${DateTimeUtils.MILLIS_PER_DAY}; + |} + |""".stripMargin + reusablePerRecordStatements.add(field) + fieldTerm + } + + /** + * Adds a reusable local time to the beginning of the SAM of the generated [[Function]]. + */ + def addReusableLocalTime(): String = { + val fieldTerm = s"localtime" + + val localtimestamp = addReusableLocalTimestamp() + + // adopted from org.apache.calcite.runtime.SqlFunctions.localTime() + val field = + s""" + |final int $fieldTerm = (int) ($localtimestamp % ${DateTimeUtils.MILLIS_PER_DAY}); + |""".stripMargin + reusablePerRecordStatements.add(field) + fieldTerm + } + + + /** + * Adds a reusable date to the beginning of the SAM of the generated [[Function]]. + */ + def addReusableDate(): String = { + val fieldTerm = s"date" + + val timestamp = addReusableTimestamp() + val time = addReusableTime() + + // adopted from org.apache.calcite.runtime.SqlFunctions.currentDate() + val field = + s""" + |final int $fieldTerm = (int) ($timestamp / ${DateTimeUtils.MILLIS_PER_DAY}); + |if ($time < 0) { + | $fieldTerm -= 1; + |} + |""".stripMargin + reusablePerRecordStatements.add(field) + fieldTerm + } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/CurrentTimePointCallGen.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/CurrentTimePointCallGen.scala new file mode 100644 index 0000000000000..4aaa209db3129 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/CurrentTimePointCallGen.scala @@ -0,0 +1,58 @@ +/* + * 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.table.codegen.calls + +import org.apache.flink.api.common.typeinfo.{SqlTimeTypeInfo, TypeInformation} +import org.apache.flink.api.table.codegen.{CodeGenerator, GeneratedExpression} + +/** + * Generates function call to determine current time point (as date/time/timestamp) in + * local timezone or not. + */ +class CurrentTimePointCallGen( + targetType: TypeInformation[_], + local: Boolean) + extends CallGenerator { + + override def generate( + codeGenerator: CodeGenerator, + operands: Seq[GeneratedExpression]) + : GeneratedExpression = targetType match { + case SqlTimeTypeInfo.TIME if local => + val time = codeGenerator.addReusableLocalTime() + codeGenerator.generateNonNullLiteral(targetType, time) + + case SqlTimeTypeInfo.TIMESTAMP if local => + val timestamp = codeGenerator.addReusableLocalTimestamp() + codeGenerator.generateNonNullLiteral(targetType, timestamp) + + case SqlTimeTypeInfo.DATE => + val date = codeGenerator.addReusableDate() + codeGenerator.generateNonNullLiteral(targetType, date) + + case SqlTimeTypeInfo.TIME => + val time = codeGenerator.addReusableTime() + codeGenerator.generateNonNullLiteral(targetType, time) + + case SqlTimeTypeInfo.TIMESTAMP => + val timestamp = codeGenerator.addReusableTimestamp() + codeGenerator.generateNonNullLiteral(targetType, timestamp) + } + +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/ScalarFunctions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/ScalarFunctions.scala index 8aa632f805250..24e8290317d88 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/ScalarFunctions.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/ScalarFunctions.scala @@ -223,6 +223,31 @@ object ScalarFunctions { BuiltInMethod.CEIL.method, Some(BuiltInMethod.UNIX_TIMESTAMP_CEIL.method))) + addSqlFunction( + CURRENT_DATE, + Seq(), + new CurrentTimePointCallGen(SqlTimeTypeInfo.DATE, local = false)) + + addSqlFunction( + CURRENT_TIME, + Seq(), + new CurrentTimePointCallGen(SqlTimeTypeInfo.TIME, local = false)) + + addSqlFunction( + CURRENT_TIMESTAMP, + Seq(), + new CurrentTimePointCallGen(SqlTimeTypeInfo.TIMESTAMP, local = false)) + + addSqlFunction( + LOCALTIME, + Seq(), + new CurrentTimePointCallGen(SqlTimeTypeInfo.TIME, local = true)) + + addSqlFunction( + LOCALTIMESTAMP, + Seq(), + new CurrentTimePointCallGen(SqlTimeTypeInfo.TIMESTAMP, local = true)) + // ---------------------------------------------------------------------------------------------- /** diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/time.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/time.scala index 48b512cd1e634..385b3d5be3e03 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/time.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/time.scala @@ -201,3 +201,50 @@ case class TemporalCeil( } } +abstract class CurrentTimePoint( + targetType: TypeInformation[_], + local: Boolean) + extends LeafExpression { + + override private[flink] def resultType: TypeInformation[_] = targetType + + override private[flink] def validateInput(): ExprValidationResult = { + if (!TypeCheckUtils.isTimePoint(targetType)) { + ValidationFailure(s"CurrentTimePoint operator requires Time Point target type, " + + s"but get $targetType.") + } else if (local && targetType == SqlTimeTypeInfo.DATE) { + ValidationFailure(s"Localized CurrentTimePoint operator requires Time or Timestamp target " + + s"type, but get $targetType.") + } else { + ValidationSuccess + } + } + + override def toString: String = if (local) { + s"local$targetType()" + } else { + s"current$targetType()" + } + + override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = { + val operator = targetType match { + case SqlTimeTypeInfo.TIME if local => SqlStdOperatorTable.LOCALTIME + case SqlTimeTypeInfo.TIMESTAMP if local => SqlStdOperatorTable.LOCALTIMESTAMP + case SqlTimeTypeInfo.DATE => SqlStdOperatorTable.CURRENT_DATE + case SqlTimeTypeInfo.TIME => SqlStdOperatorTable.CURRENT_TIME + case SqlTimeTypeInfo.TIMESTAMP => SqlStdOperatorTable.CURRENT_TIMESTAMP + } + relBuilder.call(operator) + } +} + +case class CurrentDate() extends CurrentTimePoint(SqlTimeTypeInfo.DATE, local = false) + +case class CurrentTime() extends CurrentTimePoint(SqlTimeTypeInfo.TIME, local = false) + +case class CurrentTimestamp() extends CurrentTimePoint(SqlTimeTypeInfo.TIMESTAMP, local = false) + +case class LocalTime() extends CurrentTimePoint(SqlTimeTypeInfo.TIME, local = true) + +case class LocalTimestamp() extends CurrentTimePoint(SqlTimeTypeInfo.TIMESTAMP, local = true) + diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/validate/FunctionCatalog.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/validate/FunctionCatalog.scala index b9a3f71c92725..98086728f0c38 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/validate/FunctionCatalog.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/validate/FunctionCatalog.scala @@ -151,7 +151,13 @@ object FunctionCatalog { "mod" -> classOf[Mod], // temporal functions - "extract" -> classOf[Extract] + "extract" -> classOf[Extract], + "currentDate" -> classOf[CurrentDate], + "currentTime" -> classOf[CurrentTime], + "currentTimestamp" -> classOf[CurrentTimestamp], + "localTime" -> classOf[LocalTime], + "localTimestamp" -> classOf[LocalTimestamp] + // TODO implement function overloading here // "floor" -> classOf[TemporalFloor] // "ceil" -> classOf[TemporalCeil] diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/NonDeterministicTests.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/NonDeterministicTests.scala new file mode 100644 index 0000000000000..de48849159d13 --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/NonDeterministicTests.scala @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.table.expressions + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.scala.table._ +import org.apache.flink.api.table.Row +import org.apache.flink.api.table.expressions.utils.ExpressionTestBase +import org.apache.flink.api.table.typeutils.RowTypeInfo +import org.junit.{Ignore, Test} + +/** + * Tests that can only be checked manually as they are non-deterministic. + */ +class NonDeterministicTests extends ExpressionTestBase { + + @Ignore + @Test + def testCurrentDate(): Unit = { + testAllApis( + currentDate(), + "currentDate()", + "CURRENT_DATE", + "PLEASE CHECK MANUALLY") + } + + @Ignore + @Test + def testCurrentTime(): Unit = { + testAllApis( + currentTime(), + "currentTime()", + "CURRENT_TIME", + "PLEASE CHECK MANUALLY") + } + + @Ignore + @Test + def testCurrentTimestamp(): Unit = { + testAllApis( + currentTimestamp(), + "currentTimestamp()", + "CURRENT_TIMESTAMP", + "PLEASE CHECK MANUALLY") + } + + @Ignore + @Test + def testLocalTimestamp(): Unit = { + testAllApis( + localTimestamp(), + "localTimestamp()", + "LOCALTIMESTAMP", + "PLEASE CHECK MANUALLY") + } + + @Ignore + @Test + def testLocalTime(): Unit = { + testAllApis( + localTime(), + "localTime()", + "LOCALTIME", + "PLEASE CHECK MANUALLY") + } + + // ---------------------------------------------------------------------------------------------- + + override def testData: Any = new Row(0) + + override def typeInfo: TypeInformation[Any] = + new RowTypeInfo(Seq[TypeInformation[_]]()).asInstanceOf[TypeInformation[Any]] +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarFunctionsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarFunctionsTest.scala index 7ab0c7df6ae11..516bfcaa93281 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarFunctionsTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarFunctionsTest.scala @@ -753,6 +753,51 @@ class ScalarFunctionsTest extends ExpressionTestBase { "true") } + @Test + def testCurrentTimePoint(): Unit = { + + // current time points are non-deterministic + // we just test the format of the output + // manual test can be found in NonDeterministicTests + + testAllApis( + currentDate().cast(Types.STRING).charLength(), + "currentDate().cast(STRING).charLength()", + "CHAR_LENGTH(CAST(CURRENT_DATE AS VARCHAR))", + "10") + + testAllApis( + currentTime().cast(Types.STRING).charLength(), + "currentTime().cast(STRING).charLength()", + "CHAR_LENGTH(CAST(CURRENT_TIME AS VARCHAR))", + "8") + + testAllApis( + currentTimestamp().cast(Types.STRING).charLength() >= 22, + "currentTimestamp().cast(STRING).charLength() >= 22", + "CHAR_LENGTH(CAST(CURRENT_TIMESTAMP AS VARCHAR)) >= 22", + "true") + + testAllApis( + localTimestamp().cast(Types.STRING).charLength() >= 22, + "localTimestamp().cast(STRING).charLength() >= 22", + "CHAR_LENGTH(CAST(LOCALTIMESTAMP AS VARCHAR)) >= 22", + "true") + + testAllApis( + localTime().cast(Types.STRING).charLength(), + "localTime().cast(STRING).charLength()", + "CHAR_LENGTH(CAST(LOCALTIME AS VARCHAR))", + "8") + + // comparisons are deterministic + testAllApis( + localTimestamp() === localTimestamp(), + "localTimestamp() === localTimestamp()", + "LOCALTIMESTAMP = LOCALTIMESTAMP", + "true") + } + // ---------------------------------------------------------------------------------------------- def testData = { From 15a8a1ae04a72cc9572d0c7dd156de1083be9e82 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Fri, 2 Sep 2016 17:52:51 +0200 Subject: [PATCH 021/299] [FLINK-4570] remove conflicting Unicode character MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This caused Scalastyle to fail, presumably depending on the locale used. After a bit of debugging on the Scalastyle plugin I found out that the number in the error is the byte position. "Expected identifier, but got Token(COMMA,,,1772,,)" head -c 1772 flink-mesos/src/test/scala/org/apache/flink/mesos/Utils.scala pointed to the Unicode character '⇒' which causes Scalastyle to fail in certain environments. This closes #2466 --- .../src/test/scala/org/apache/flink/mesos/Utils.scala | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/flink-mesos/src/test/scala/org/apache/flink/mesos/Utils.scala b/flink-mesos/src/test/scala/org/apache/flink/mesos/Utils.scala index 6f27795fbfb1e..bd6235a8d2e73 100644 --- a/flink-mesos/src/test/scala/org/apache/flink/mesos/Utils.scala +++ b/flink-mesos/src/test/scala/org/apache/flink/mesos/Utils.scala @@ -16,14 +16,12 @@ * limitations under the License. */ -// disable Scalastyle for now to prevent random failures reported in FLINK-4570 -// scalastyle:off package org.apache.flink.mesos import java.util.concurrent.atomic.AtomicLong import akka.actor._ -import akka.testkit.{TestActorRef, TestFSMRef} +import akka.testkit.TestFSMRef import org.mockito.ArgumentMatcher import scala.collection.JavaConverters._ @@ -49,9 +47,8 @@ object TestFSMUtils { "$" + akka.util.Helpers.base64(l) } - def testFSMRef[S, D, T <: Actor: ClassTag](factory: ⇒ T, supervisor: ActorRef) + def testFSMRef[S, D, T <: Actor: ClassTag](factory: => T, supervisor: ActorRef) (implicit ev: T <:< FSM[S, D], system: ActorSystem): TestFSMRef[S, D, T] = { new TestFSMRef(system, Props(factory), supervisor, TestFSMUtils.randomName) } } -// scalastyle:on From 2e6e3dc96c8bc70325f3e2829752693158b18925 Mon Sep 17 00:00:00 2001 From: Kurt Young Date: Fri, 26 Aug 2016 17:51:40 +0800 Subject: [PATCH 022/299] [FLINK-4459] [distributed runtime] Introduce SlotProvider for Scheduler This closes #2424 --- .../runtime/executiongraph/Execution.java | 15 +- .../executiongraph/ExecutionGraph.java | 32 +- .../executiongraph/ExecutionJobVertex.java | 7 +- .../executiongraph/ExecutionVertex.java | 8 +- .../flink/runtime/instance/SlotProvider.java | 48 +++ .../jobmanager/scheduler/Scheduler.java | 27 +- .../ExecutionGraphMetricsTest.java | 8 +- .../ExecutionVertexSchedulingTest.java | 28 +- .../TerminalStateDeadlockTest.java | 27 +- .../ScheduleWithCoLocationHintTest.java | 303 ++++++++++-------- .../scheduler/SchedulerIsolatedTasksTest.java | 45 ++- .../scheduler/SchedulerSlotSharingTest.java | 230 ++++++------- 12 files changed, 430 insertions(+), 348 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotProvider.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java index 846df4956d854..68263654781b4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java @@ -34,13 +34,13 @@ import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.instance.SimpleSlot; +import org.apache.flink.runtime.instance.SlotProvider; import org.apache.flink.runtime.io.network.ConnectionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobmanager.scheduler.CoLocationConstraint; import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException; import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit; -import org.apache.flink.runtime.jobmanager.scheduler.Scheduler; import org.apache.flink.runtime.jobmanager.scheduler.SlotAllocationFuture; import org.apache.flink.runtime.jobmanager.scheduler.SlotAllocationFutureAction; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; @@ -271,15 +271,15 @@ public void setInitialState( * to be scheduled immediately and no resource is available. If the task is accepted by the schedule, any * error sets the vertex state to failed and triggers the recovery logic. * - * @param scheduler The scheduler to use to schedule this execution attempt. + * @param slotProvider The slot provider to use to allocate slot for this execution attempt. * @param queued Flag to indicate whether the scheduler may queue this task if it cannot * immediately deploy it. * * @throws IllegalStateException Thrown, if the vertex is not in CREATED state, which is the only state that permits scheduling. * @throws NoResourceAvailableException Thrown is no queued scheduling is allowed and no resources are currently available. */ - public boolean scheduleForExecution(Scheduler scheduler, boolean queued) throws NoResourceAvailableException { - if (scheduler == null) { + public boolean scheduleForExecution(SlotProvider slotProvider, boolean queued) throws NoResourceAvailableException { + if (slotProvider == null) { throw new IllegalArgumentException("Cannot send null Scheduler when scheduling execution."); } @@ -299,9 +299,8 @@ public boolean scheduleForExecution(Scheduler scheduler, boolean queued) throws // IMPORTANT: To prevent leaks of cluster resources, we need to make sure that slots are returned // in all cases where the deployment failed. we use many try {} finally {} clauses to assure that + final SlotAllocationFuture future = slotProvider.allocateSlot(toSchedule, queued); if (queued) { - SlotAllocationFuture future = scheduler.scheduleQueued(toSchedule); - future.setFutureAction(new SlotAllocationFutureAction() { @Override public void slotAllocated(SimpleSlot slot) { @@ -319,7 +318,7 @@ public void slotAllocated(SimpleSlot slot) { }); } else { - SimpleSlot slot = scheduler.scheduleImmediately(toSchedule); + SimpleSlot slot = future.get(); try { deployToSlot(slot); } @@ -560,7 +559,7 @@ else if (numConsumers == 0) { public Boolean call() throws Exception { try { consumerVertex.scheduleForExecution( - consumerVertex.getExecutionGraph().getScheduler(), + consumerVertex.getExecutionGraph().getSlotProvider(), consumerVertex.getExecutionGraph().isQueuedSchedulingAllowed()); } catch (Throwable t) { consumerVertex.fail(new IllegalStateException("Could not schedule consumer " + diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index 92cab41cf5f48..585e9f3a6eec7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -40,6 +40,7 @@ import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.execution.SuppressRestartsException; import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; +import org.apache.flink.runtime.instance.SlotProvider; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.JobStatus; @@ -47,7 +48,6 @@ import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.ScheduleMode; import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; -import org.apache.flink.runtime.jobmanager.scheduler.Scheduler; import org.apache.flink.runtime.query.KvStateLocationRegistry; import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.runtime.util.SerializableObject; @@ -197,8 +197,8 @@ public class ExecutionGraph { // ------ Fields that are relevant to the execution and need to be cleared before archiving ------- - /** The scheduler to use for scheduling new tasks as they are needed */ - private Scheduler scheduler; + /** The slot provider to use for allocating slots for tasks as they are needed */ + private SlotProvider slotProvider; /** Strategy to use for restarts */ private RestartStrategy restartStrategy; @@ -470,8 +470,8 @@ public String getJsonPlan() { return jsonPlan; } - public Scheduler getScheduler() { - return scheduler; + public SlotProvider getSlotProvider() { + return slotProvider; } public JobID getJobID() { @@ -670,17 +670,17 @@ public void attachJobGraph(List topologiallySorted) throws JobExcepti } } - public void scheduleForExecution(Scheduler scheduler) throws JobException { - if (scheduler == null) { + public void scheduleForExecution(SlotProvider slotProvider) throws JobException { + if (slotProvider == null) { throw new IllegalArgumentException("Scheduler must not be null."); } - if (this.scheduler != null && this.scheduler != scheduler) { - throw new IllegalArgumentException("Cannot use different schedulers for the same job"); + if (this.slotProvider != null && this.slotProvider != slotProvider) { + throw new IllegalArgumentException("Cannot use different slot providers for the same job"); } if (transitionState(JobStatus.CREATED, JobStatus.RUNNING)) { - this.scheduler = scheduler; + this.slotProvider = slotProvider; switch (scheduleMode) { @@ -688,14 +688,14 @@ public void scheduleForExecution(Scheduler scheduler) throws JobException { // simply take the vertices without inputs. for (ExecutionJobVertex ejv : this.tasks.values()) { if (ejv.getJobVertex().isInputVertex()) { - ejv.scheduleAll(scheduler, allowQueuedScheduling); + ejv.scheduleAll(slotProvider, allowQueuedScheduling); } } break; case EAGER: for (ExecutionJobVertex ejv : getVerticesTopologically()) { - ejv.scheduleAll(scheduler, allowQueuedScheduling); + ejv.scheduleAll(slotProvider, allowQueuedScheduling); } break; @@ -850,8 +850,8 @@ public void restart() { throw new IllegalStateException("Can only restart job from state restarting."); } - if (scheduler == null) { - throw new IllegalStateException("The execution graph has not been scheduled before - scheduler is null."); + if (slotProvider == null) { + throw new IllegalStateException("The execution graph has not been scheduled before - slotProvider is null."); } this.currentExecutions.clear(); @@ -885,7 +885,7 @@ public void restart() { } } - scheduleForExecution(scheduler); + scheduleForExecution(slotProvider); } catch (Throwable t) { fail(t); @@ -917,7 +917,7 @@ public void prepareForArchiving() { // clear the non-serializable fields restartStrategy = null; - scheduler = null; + slotProvider = null; checkpointCoordinator = null; executionContext = null; kvStateLocationRegistry = null; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java index d3dc8fe46e144..1ac9522bb4a8a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java @@ -29,6 +29,7 @@ import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult; import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.instance.SlotProvider; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.IntermediateDataSet; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; @@ -37,7 +38,6 @@ import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException; -import org.apache.flink.runtime.jobmanager.scheduler.Scheduler; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; import org.apache.flink.runtime.util.SerializableObject; import org.apache.flink.util.Preconditions; @@ -289,12 +289,13 @@ public void connectToPredecessors(Map // Actions //--------------------------------------------------------------------------------------------- - public void scheduleAll(Scheduler scheduler, boolean queued) throws NoResourceAvailableException { + public void scheduleAll(SlotProvider slotProvider, boolean queued) throws NoResourceAvailableException { + ExecutionVertex[] vertices = this.taskVertices; // kick off the tasks for (ExecutionVertex ev : vertices) { - ev.scheduleForExecution(scheduler, queued); + ev.scheduleForExecution(slotProvider, queued); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java index 88e1b889c7ea6..a8d5ee4a3c7b9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java @@ -27,6 +27,7 @@ import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.instance.SlotProvider; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.instance.SimpleSlot; @@ -40,12 +41,11 @@ import org.apache.flink.runtime.jobmanager.scheduler.CoLocationConstraint; import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException; -import org.apache.flink.runtime.jobmanager.scheduler.Scheduler; import org.apache.flink.util.SerializedValue; - import org.apache.flink.runtime.state.ChainedStateHandle; import org.apache.flink.runtime.state.KeyGroupsStateHandle; import org.apache.flink.runtime.state.StreamStateHandle; + import org.slf4j.Logger; import scala.concurrent.duration.FiniteDuration; @@ -443,8 +443,8 @@ public void resetForNewExecution() { } } - public boolean scheduleForExecution(Scheduler scheduler, boolean queued) throws NoResourceAvailableException { - return this.currentExecution.scheduleForExecution(scheduler, queued); + public boolean scheduleForExecution(SlotProvider slotProvider, boolean queued) throws NoResourceAvailableException { + return this.currentExecution.scheduleForExecution(slotProvider, queued); } public void deployToSlot(SimpleSlot slot) throws JobException { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotProvider.java new file mode 100644 index 0000000000000..b2c23a5d9195c --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotProvider.java @@ -0,0 +1,48 @@ +/* + * 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.runtime.instance; + +import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException; +import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit; +import org.apache.flink.runtime.jobmanager.scheduler.SlotAllocationFuture; + +/** + * The slot provider is responsible for preparing slots for ready-to-run tasks. + * + *

It supports two allocating modes: + *

    + *
  • Immediate allocating: A request for a task slot immediately gets satisfied, we can call + * {@link SlotAllocationFuture#get()} to get the allocated slot.
  • + *
  • Queued allocating: A request for a task slot is queued and returns a future that will be + * fulfilled as soon as a slot becomes available.
  • + *
+ */ +public interface SlotProvider { + + /** + * Allocating slot with specific requirement. + * + * @param task The task to allocate the slot for + * @param allowQueued Whether allow the task be queued if we do not have enough resource + * @return The future of the allocation + * + * @throws NoResourceAvailableException + */ + SlotAllocationFuture allocateSlot(ScheduledUnit task, boolean allowQueued) throws NoResourceAvailableException; +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java index 734972dd8dc72..c9cdd00be5678 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java @@ -39,6 +39,7 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.instance.SlotProvider; import org.apache.flink.runtime.instance.SlotSharingGroupAssignment; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.instance.SharedSlot; @@ -65,7 +66,7 @@ * fulfilled as soon as a slot becomes available. * */ -public class Scheduler implements InstanceListener, SlotAvailabilityListener { +public class Scheduler implements InstanceListener, SlotAvailabilityListener, SlotProvider { /** Scheduler-wide logger */ private static final Logger LOG = LoggerFactory.getLogger(Scheduler.class); @@ -129,30 +130,24 @@ public void shutdown() { // ------------------------------------------------------------------------ // Scheduling // ------------------------------------------------------------------------ - - public SimpleSlot scheduleImmediately(ScheduledUnit task) throws NoResourceAvailableException { - Object ret = scheduleTask(task, false); - if (ret instanceof SimpleSlot) { - return (SimpleSlot) ret; - } - else { - throw new RuntimeException(); - } - } - - public SlotAllocationFuture scheduleQueued(ScheduledUnit task) throws NoResourceAvailableException { - Object ret = scheduleTask(task, true); + + + @Override + public SlotAllocationFuture allocateSlot(ScheduledUnit task, boolean allowQueued) + throws NoResourceAvailableException { + + final Object ret = scheduleTask(task, allowQueued); if (ret instanceof SimpleSlot) { return new SlotAllocationFuture((SimpleSlot) ret); } - if (ret instanceof SlotAllocationFuture) { + else if (ret instanceof SlotAllocationFuture) { return (SlotAllocationFuture) ret; } else { throw new RuntimeException(); } } - + /** * Returns either a {@link org.apache.flink.runtime.instance.SimpleSlot}, or a {@link SlotAllocationFuture}. */ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java index d5520fddaf590..aa5925f8a735b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java @@ -33,6 +33,7 @@ import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.instance.Instance; +import org.apache.flink.runtime.jobmanager.scheduler.SlotAllocationFuture; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.instance.Slot; @@ -70,8 +71,8 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; + +import static org.mockito.Mockito.*; public class ExecutionGraphMetricsTest extends TestLogger { @@ -135,7 +136,8 @@ public void testExecutionGraphRestartTimeMetric() throws JobException, IOExcepti when(simpleSlot.setExecutedVertex(Matchers.any(Execution.class))).thenReturn(true); when(simpleSlot.getRoot()).thenReturn(rootSlot); - when(scheduler.scheduleImmediately(Matchers.any(ScheduledUnit.class))).thenReturn(simpleSlot); + when(scheduler.allocateSlot(any(ScheduledUnit.class), anyBoolean())) + .thenReturn(new SlotAllocationFuture(simpleSlot)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java index 5e9ee3398400e..c576ce5d39216 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java @@ -18,25 +18,29 @@ package org.apache.flink.runtime.executiongraph; -import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getExecutionVertex; -import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getInstance; -import static org.junit.Assert.*; -import static org.mockito.Mockito.*; - import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.instance.DummyActorGateway; import org.apache.flink.runtime.instance.Instance; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.runtime.jobmanager.scheduler.Scheduler; import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit; +import org.apache.flink.runtime.jobmanager.scheduler.Scheduler; import org.apache.flink.runtime.jobmanager.scheduler.SlotAllocationFuture; import org.apache.flink.runtime.testingUtils.TestingUtils; -import org.junit.Test; +import org.junit.Test; import org.mockito.Matchers; +import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getExecutionVertex; +import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getInstance; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Mockito.anyBoolean; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + public class ExecutionVertexSchedulingTest { @Test @@ -54,7 +58,8 @@ public void testSlotReleasedWhenScheduledImmediately() { assertTrue(slot.isReleased()); Scheduler scheduler = mock(Scheduler.class); - when(scheduler.scheduleImmediately(Matchers.any(ScheduledUnit.class))).thenReturn(slot); + when(scheduler.allocateSlot(Matchers.any(ScheduledUnit.class), anyBoolean())) + .thenReturn(new SlotAllocationFuture(slot)); assertEquals(ExecutionState.CREATED, vertex.getExecutionState()); // try to deploy to the slot @@ -86,7 +91,7 @@ public void testSlotReleasedWhenScheduledQueued() { final SlotAllocationFuture future = new SlotAllocationFuture(); Scheduler scheduler = mock(Scheduler.class); - when(scheduler.scheduleQueued(Matchers.any(ScheduledUnit.class))).thenReturn(future); + when(scheduler.allocateSlot(Matchers.any(ScheduledUnit.class), anyBoolean())).thenReturn(future); assertEquals(ExecutionState.CREATED, vertex.getExecutionState()); // try to deploy to the slot @@ -117,7 +122,8 @@ public void testScheduleToDeploying() { final SimpleSlot slot = instance.allocateSimpleSlot(ejv.getJobId()); Scheduler scheduler = mock(Scheduler.class); - when(scheduler.scheduleImmediately(Matchers.any(ScheduledUnit.class))).thenReturn(slot); + when(scheduler.allocateSlot(Matchers.any(ScheduledUnit.class), anyBoolean())) + .thenReturn(new SlotAllocationFuture(slot)); assertEquals(ExecutionState.CREATED, vertex.getExecutionState()); @@ -130,4 +136,4 @@ public void testScheduleToDeploying() { fail(e.getMessage()); } } -} \ No newline at end of file +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java index 870ae05bf3c1e..4cae7c2f48a72 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java @@ -30,6 +30,7 @@ import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.instance.SimpleSlot; +import org.apache.flink.runtime.instance.SlotProvider; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.JobVertexID; @@ -54,12 +55,12 @@ import static org.junit.Assert.*; public class TerminalStateDeadlockTest { - + private final Field stateField; private final Field resourceField; private final Field execGraphStateField; - private final Field execGraphSchedulerField; - + private final Field execGraphSlotProviderField; + private final SimpleSlot resource; @@ -75,8 +76,8 @@ public TerminalStateDeadlockTest() { this.execGraphStateField = ExecutionGraph.class.getDeclaredField("state"); this.execGraphStateField.setAccessible(true); - this.execGraphSchedulerField = ExecutionGraph.class.getDeclaredField("scheduler"); - this.execGraphSchedulerField.setAccessible(true); + this.execGraphSlotProviderField = ExecutionGraph.class.getDeclaredField("slotProvider"); + this.execGraphSlotProviderField.setAccessible(true); // the dummy resource ResourceID resourceId = ResourceID.generate(); @@ -96,11 +97,9 @@ public TerminalStateDeadlockTest() { throw new RuntimeException(); } } - - - + // ------------------------------------------------------------------------ - + @Test public void testProvokeDeadlock() { try { @@ -135,7 +134,7 @@ public void testProvokeDeadlock() { initializeExecution(e2); execGraphStateField.set(eg, JobStatus.FAILING); - execGraphSchedulerField.set(eg, scheduler); + execGraphSlotProviderField.set(eg, scheduler); Runnable r1 = new Runnable() { @Override @@ -173,12 +172,10 @@ private void initializeExecution(Execution exec) throws IllegalAccessException { static class TestExecGraph extends ExecutionGraph { - private static final long serialVersionUID = -7606144898417942044L; - private static final Configuration EMPTY_CONFIG = new Configuration(); private static final FiniteDuration TIMEOUT = new FiniteDuration(30, TimeUnit.SECONDS); - + private volatile boolean done; TestExecGraph(JobID jobId) throws IOException { @@ -193,14 +190,14 @@ static class TestExecGraph extends ExecutionGraph { } @Override - public void scheduleForExecution(Scheduler scheduler) { + public void scheduleForExecution(SlotProvider slotProvider) { // notify that we are done with the "restarting" synchronized (this) { done = true; this.notifyAll(); } } - + public void waitTillDone() { try { synchronized (this) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java index eab4fea69a372..b8037023172e8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java @@ -63,18 +63,18 @@ public void scheduleAllSharedAndCoLocated() { CoLocationConstraint c6 = new CoLocationConstraint(ccg); // schedule 4 tasks from the first vertex group - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 6), sharingGroup, c1)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 1, 6), sharingGroup, c2)); - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 2, 6), sharingGroup, c3)); - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 3, 6), sharingGroup, c4)); - SimpleSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 6), sharingGroup, c1)); - SimpleSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 1, 6), sharingGroup, c2)); - SimpleSlot s7 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 2, 6), sharingGroup, c3)); - SimpleSlot s8 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 4, 6), sharingGroup, c5)); - SimpleSlot s9 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 5, 6), sharingGroup, c6)); - SimpleSlot s10 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 3, 6), sharingGroup, c4)); - SimpleSlot s11 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 4, 6), sharingGroup, c5)); - SimpleSlot s12 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 5, 6), sharingGroup, c6)); + SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 6), sharingGroup, c1), false).get(); + SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 6), sharingGroup, c2), false).get(); + SimpleSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 6), sharingGroup, c3), false).get(); + SimpleSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 6), sharingGroup, c4), false).get(); + SimpleSlot s5 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 6), sharingGroup, c1), false).get(); + SimpleSlot s6 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 6), sharingGroup, c2), false).get(); + SimpleSlot s7 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 6), sharingGroup, c3), false).get(); + SimpleSlot s8 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 4, 6), sharingGroup, c5), false).get(); + SimpleSlot s9 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 5, 6), sharingGroup, c6), false).get(); + SimpleSlot s10 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 3, 6), sharingGroup, c4), false).get(); + SimpleSlot s11 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 4, 6), sharingGroup, c5), false).get(); + SimpleSlot s12 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 5, 6), sharingGroup, c6), false).get(); assertNotNull(s1); assertNotNull(s2); @@ -109,22 +109,22 @@ public void scheduleAllSharedAndCoLocated() { assertEquals(s4.getTaskManagerID(), s10.getTaskManagerID()); assertEquals(s8.getTaskManagerID(), s11.getTaskManagerID()); assertEquals(s9.getTaskManagerID(), s12.getTaskManagerID()); - + assertEquals(c1.getLocation(), s1.getTaskManagerLocation()); assertEquals(c2.getLocation(), s2.getTaskManagerLocation()); assertEquals(c3.getLocation(), s3.getTaskManagerLocation()); assertEquals(c4.getLocation(), s4.getTaskManagerLocation()); assertEquals(c5.getLocation(), s8.getTaskManagerLocation()); assertEquals(c6.getLocation(), s9.getTaskManagerLocation()); - + // check the scheduler's bookkeeping assertEquals(0, scheduler.getNumberOfAvailableSlots()); - + // the first assignments are unconstrained, co.-scheduling is constrained assertEquals(6, scheduler.getNumberOfLocalizedAssignments()); assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments()); assertEquals(6, scheduler.getNumberOfUnconstrainedAssignments()); - + // release some slots, be sure that new available ones come up s1.releaseSlot(); s2.releaseSlot(); @@ -135,10 +135,11 @@ public void scheduleAllSharedAndCoLocated() { s11.releaseSlot(); s12.releaseSlot(); assertTrue(scheduler.getNumberOfAvailableSlots() >= 1); - - SimpleSlot single = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(new JobVertexID(), 0, 1))); + + SimpleSlot single = scheduler.allocateSlot( + new ScheduledUnit(getTestVertex(new JobVertexID(), 0, 1)), false).get(); assertNotNull(single); - + s1.releaseSlot(); s2.releaseSlot(); s3.releaseSlot(); @@ -149,9 +150,9 @@ public void scheduleAllSharedAndCoLocated() { s9.releaseSlot(); s11.releaseSlot(); s12.releaseSlot(); - + assertEquals(5, scheduler.getNumberOfAvailableSlots()); - + assertEquals(6, scheduler.getNumberOfLocalizedAssignments()); assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments()); assertEquals(7, scheduler.getNumberOfUnconstrainedAssignments()); @@ -161,7 +162,7 @@ public void scheduleAllSharedAndCoLocated() { fail(e.getMessage()); } } - + @Test public void scheduleWithIntermediateRelease() { try { @@ -169,34 +170,37 @@ public void scheduleWithIntermediateRelease() { JobVertexID jid2 = new JobVertexID(); JobVertexID jid3 = new JobVertexID(); JobVertexID jid4 = new JobVertexID(); - + Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext()); - + Instance i1 = getRandomInstance(1); Instance i2 = getRandomInstance(1); - + scheduler.newInstanceAvailable(i1); scheduler.newInstanceAvailable(i2); - + assertEquals(2, scheduler.getNumberOfAvailableSlots()); - + SlotSharingGroup sharingGroup = new SlotSharingGroup(); CoLocationConstraint c1 = new CoLocationConstraint(new CoLocationGroup()); - - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 1), sharingGroup, c1)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 1), sharingGroup, c1)); - - SimpleSlot sSolo = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 0, 1))); - + + SimpleSlot s1 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertex(jid1, 0, 1), sharingGroup, c1), false).get(); + SimpleSlot s2 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertex(jid2, 0, 1), sharingGroup, c1), false).get(); + + SimpleSlot sSolo = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 0, 1)), false).get(); + ResourceID taskManager = s1.getTaskManagerID(); - + s1.releaseSlot(); s2.releaseSlot(); sSolo.releaseSlot(); - - SimpleSlot sNew = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 0, 1), sharingGroup, c1)); + + SimpleSlot sNew = scheduler.allocateSlot( + new ScheduledUnit(getTestVertex(jid3, 0, 1), sharingGroup, c1), false).get(); assertEquals(taskManager, sNew.getTaskManagerID()); - + assertEquals(2, scheduler.getNumberOfLocalizedAssignments()); assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments()); assertEquals(2, scheduler.getNumberOfUnconstrainedAssignments()); @@ -206,41 +210,41 @@ public void scheduleWithIntermediateRelease() { fail(e.getMessage()); } } - + @Test public void scheduleWithReleaseNoResource() { try { JobVertexID jid1 = new JobVertexID(); JobVertexID jid2 = new JobVertexID(); JobVertexID jid3 = new JobVertexID(); - + Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext()); - + Instance i1 = getRandomInstance(1); Instance i2 = getRandomInstance(1); - + scheduler.newInstanceAvailable(i1); scheduler.newInstanceAvailable(i2); - + assertEquals(2, scheduler.getNumberOfAvailableSlots()); - + SlotSharingGroup sharingGroup = new SlotSharingGroup(); CoLocationConstraint c1 = new CoLocationConstraint(new CoLocationGroup()); - - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 1), sharingGroup, c1)); + + SimpleSlot s1 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertex(jid1, 0, 1), sharingGroup, c1), false).get(); s1.releaseSlot(); - - scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 1))); - scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 1, 2))); - - + + scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 1)), false).get(); + scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 2)), false).get(); + try { - scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 0, 1), sharingGroup, c1)); + scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 0, 1), sharingGroup, c1), false).get(); fail("Scheduled even though no resource was available."); } catch (NoResourceAvailableException e) { // expected } - + assertEquals(0, scheduler.getNumberOfLocalizedAssignments()); assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments()); assertEquals(3, scheduler.getNumberOfUnconstrainedAssignments()); @@ -250,7 +254,7 @@ public void scheduleWithReleaseNoResource() { fail(e.getMessage()); } } - + @Test public void scheduleMixedCoLocationSlotSharing() { try { @@ -276,27 +280,35 @@ public void scheduleMixedCoLocationSlotSharing() { SlotSharingGroup shareGroup = new SlotSharingGroup(); // first wave - scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 4), shareGroup)); - scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 2, 4), shareGroup)); - scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 1, 4), shareGroup)); - scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 3, 4), shareGroup)); + scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 4), shareGroup), false); + scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 4), shareGroup), false); + scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 4), shareGroup), false); + scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 4), shareGroup), false); // second wave - SimpleSlot s21 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 4), shareGroup, clc1)); - SimpleSlot s22 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 2, 4), shareGroup, clc2)); - SimpleSlot s23 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 1, 4), shareGroup, clc3)); - SimpleSlot s24 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 3, 4), shareGroup, clc4)); + SimpleSlot s21 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertex(jid2, 0, 4), shareGroup, clc1), false).get(); + SimpleSlot s22 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertex(jid2, 2, 4), shareGroup, clc2), false).get(); + SimpleSlot s23 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertex(jid2, 1, 4), shareGroup, clc3), false).get(); + SimpleSlot s24 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertex(jid2, 3, 4), shareGroup, clc4), false).get(); // third wave - SimpleSlot s31 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 1, 4), shareGroup, clc2)); - SimpleSlot s32 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 2, 4), shareGroup, clc3)); - SimpleSlot s33 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 3, 4), shareGroup, clc4)); - SimpleSlot s34 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 0, 4), shareGroup, clc1)); - - scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 0, 4), shareGroup)); - scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 1, 4), shareGroup)); - scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 2, 4), shareGroup)); - scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 3, 4), shareGroup)); + SimpleSlot s31 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertex(jid3, 1, 4), shareGroup, clc2), false).get(); + SimpleSlot s32 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertex(jid3, 2, 4), shareGroup, clc3), false).get(); + SimpleSlot s33 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertex(jid3, 3, 4), shareGroup, clc4), false).get(); + SimpleSlot s34 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertex(jid3, 0, 4), shareGroup, clc1), false).get(); + + scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 0, 4), shareGroup), false); + scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 1, 4), shareGroup), false); + scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 2, 4), shareGroup), false); + scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 3, 4), shareGroup), false); assertEquals(s21.getTaskManagerID(), s34.getTaskManagerID()); assertEquals(s22.getTaskManagerID(), s31.getTaskManagerID()); @@ -341,20 +353,26 @@ public void testGetsNonLocalFromSharingGroupFirst() { CoLocationConstraint cc2 = new CoLocationConstraint(ccg); // schedule something into the shared group so that both instances are in the sharing group - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup)); + SimpleSlot s1 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup), false).get(); + SimpleSlot s2 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup), false).get(); // schedule one locally to instance 1 - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup, cc1)); + SimpleSlot s3 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup, cc1), false).get(); // schedule with co location constraint (yet unassigned) and a preference for // instance 1, but it can only get instance 2 - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup, cc2)); + SimpleSlot s4 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup, cc2), false).get(); // schedule something into the assigned co-location constraints and check that they override the // other preferences - SimpleSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid3, 0, 2, loc2), sharingGroup, cc1)); - SimpleSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid3, 1, 2, loc1), sharingGroup, cc2)); + SimpleSlot s5 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertexWithLocation(jid3, 0, 2, loc2), sharingGroup, cc1), false).get(); + SimpleSlot s6 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertexWithLocation(jid3, 1, 2, loc1), sharingGroup, cc2), false).get(); // check that each slot got three assertEquals(3, s1.getRoot().getNumberLeaves()); @@ -386,13 +404,13 @@ public void testGetsNonLocalFromSharingGroupFirst() { fail(e.getMessage()); } } - + @Test public void testSlotReleasedInBetween() { try { JobVertexID jid1 = new JobVertexID(); JobVertexID jid2 = new JobVertexID(); - + Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext()); Instance i1 = getRandomInstance(1); @@ -403,36 +421,40 @@ public void testSlotReleasedInBetween() { scheduler.newInstanceAvailable(i2); scheduler.newInstanceAvailable(i1); - + assertEquals(2, scheduler.getNumberOfAvailableSlots()); - + SlotSharingGroup sharingGroup = new SlotSharingGroup(); - + CoLocationGroup ccg = new CoLocationGroup(); CoLocationConstraint cc1 = new CoLocationConstraint(ccg); CoLocationConstraint cc2 = new CoLocationConstraint(ccg); - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup, cc1)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup, cc2)); - + SimpleSlot s1 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup, cc1), false).get(); + SimpleSlot s2 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup, cc2), false).get(); + s1.releaseSlot(); s2.releaseSlot(); - + assertEquals(2, scheduler.getNumberOfAvailableSlots()); assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfSlots()); - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc2), sharingGroup, cc1)); - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup, cc2)); - + SimpleSlot s3 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc2), sharingGroup, cc1), false).get(); + SimpleSlot s4 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup, cc2), false).get(); + // still preserves the previous instance mapping) assertEquals(i1.getTaskManagerID(), s3.getTaskManagerID()); assertEquals(i2.getTaskManagerID(), s4.getTaskManagerID()); - + s3.releaseSlot(); s4.releaseSlot(); assertEquals(2, scheduler.getNumberOfAvailableSlots()); - + assertEquals(4, scheduler.getNumberOfLocalizedAssignments()); assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments()); assertEquals(0, scheduler.getNumberOfUnconstrainedAssignments()); @@ -442,14 +464,14 @@ public void testSlotReleasedInBetween() { fail(e.getMessage()); } } - + @Test public void testSlotReleasedInBetweenAndNoNewLocal() { try { JobVertexID jid1 = new JobVertexID(); JobVertexID jid2 = new JobVertexID(); JobVertexID jidx = new JobVertexID(); - + Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext()); Instance i1 = getRandomInstance(1); @@ -460,41 +482,46 @@ public void testSlotReleasedInBetweenAndNoNewLocal() { scheduler.newInstanceAvailable(i2); scheduler.newInstanceAvailable(i1); - + assertEquals(2, scheduler.getNumberOfAvailableSlots()); - + SlotSharingGroup sharingGroup = new SlotSharingGroup(); - + CoLocationGroup ccg = new CoLocationGroup(); CoLocationConstraint cc1 = new CoLocationConstraint(ccg); CoLocationConstraint cc2 = new CoLocationConstraint(ccg); - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup, cc1)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup, cc2)); - + SimpleSlot s1 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup, cc1), false).get(); + SimpleSlot s2 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup, cc2), false).get(); + s1.releaseSlot(); s2.releaseSlot(); - + assertEquals(2, scheduler.getNumberOfAvailableSlots()); assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfSlots()); - SimpleSlot sa = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jidx, 0, 2))); - SimpleSlot sb = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jidx, 1, 2))); - + SimpleSlot sa = scheduler.allocateSlot( + new ScheduledUnit(getTestVertexWithLocation(jidx, 0, 2)), false).get(); + SimpleSlot sb = scheduler.allocateSlot( + new ScheduledUnit(getTestVertexWithLocation(jidx, 1, 2)), false).get(); + try { - scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc2), sharingGroup, cc1)); + scheduler.allocateSlot( + new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc2), sharingGroup, cc1), false); fail("should not be able to find a resource"); } catch (NoResourceAvailableException e) { // good } catch (Exception e) { fail("wrong exception"); } - + sa.releaseSlot(); sb.releaseSlot(); assertEquals(2, scheduler.getNumberOfAvailableSlots()); - + assertEquals(2, scheduler.getNumberOfLocalizedAssignments()); assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments()); assertEquals(2, scheduler.getNumberOfUnconstrainedAssignments()); @@ -504,15 +531,15 @@ public void testSlotReleasedInBetweenAndNoNewLocal() { fail(e.getMessage()); } } - + @Test public void testScheduleOutOfOrder() { try { JobVertexID jid1 = new JobVertexID(); JobVertexID jid2 = new JobVertexID(); - + Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext()); - + Instance i1 = getRandomInstance(1); Instance i2 = getRandomInstance(1); @@ -520,11 +547,11 @@ public void testScheduleOutOfOrder() { scheduler.newInstanceAvailable(i2); scheduler.newInstanceAvailable(i1); - + assertEquals(2, scheduler.getNumberOfAvailableSlots()); - + SlotSharingGroup sharingGroup = new SlotSharingGroup(); - + CoLocationGroup ccg = new CoLocationGroup(); CoLocationConstraint cc1 = new CoLocationConstraint(ccg); CoLocationConstraint cc2 = new CoLocationConstraint(ccg); @@ -532,33 +559,37 @@ public void testScheduleOutOfOrder() { // schedule something from the second job vertex id before the first is filled, // and give locality preferences that hint at using the same shared slot for both // co location constraints (which we seek to prevent) - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup, cc1)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup, cc2)); + SimpleSlot s1 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup, cc1), false).get(); + SimpleSlot s2 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup, cc2), false).get(); + + SimpleSlot s3 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup, cc1), false).get(); + SimpleSlot s4 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc1), sharingGroup, cc2), false).get(); - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup, cc1)); - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc1), sharingGroup, cc2)); - // check that each slot got three assertEquals(2, s1.getRoot().getNumberLeaves()); assertEquals(2, s2.getRoot().getNumberLeaves()); - + assertEquals(s1.getTaskManagerID(), s3.getTaskManagerID()); assertEquals(s2.getTaskManagerID(), s4.getTaskManagerID()); - + // check the scheduler's bookkeeping assertEquals(0, scheduler.getNumberOfAvailableSlots()); - + assertEquals(3, scheduler.getNumberOfLocalizedAssignments()); assertEquals(1, scheduler.getNumberOfNonLocalizedAssignments()); assertEquals(0, scheduler.getNumberOfUnconstrainedAssignments()); - + // release some slots, be sure that new available ones come up s1.releaseSlot(); s2.releaseSlot(); s3.releaseSlot(); s4.releaseSlot(); assertEquals(2, scheduler.getNumberOfAvailableSlots()); - + assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfSlots()); assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForGroup(jid1)); assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForGroup(jid2)); @@ -568,15 +599,15 @@ public void testScheduleOutOfOrder() { fail(e.getMessage()); } } - + @Test public void nonColocationFollowsCoLocation() { try { JobVertexID jid1 = new JobVertexID(); JobVertexID jid2 = new JobVertexID(); - + Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext()); - + Instance i1 = getRandomInstance(1); Instance i2 = getRandomInstance(1); @@ -585,32 +616,36 @@ public void nonColocationFollowsCoLocation() { scheduler.newInstanceAvailable(i2); scheduler.newInstanceAvailable(i1); - + assertEquals(2, scheduler.getNumberOfAvailableSlots()); SlotSharingGroup sharingGroup = new SlotSharingGroup(); - + CoLocationGroup ccg = new CoLocationGroup(); CoLocationConstraint cc1 = new CoLocationConstraint(ccg); CoLocationConstraint cc2 = new CoLocationConstraint(ccg); - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup, cc1)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup, cc2)); + SimpleSlot s1 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup, cc1), false).get(); + SimpleSlot s2 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup, cc2), false).get(); + + SimpleSlot s3 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup), false).get(); + SimpleSlot s4 = scheduler.allocateSlot( + new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup), false).get(); - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup)); - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup)); - // check that each slot got two assertEquals(2, s1.getRoot().getNumberLeaves()); assertEquals(2, s2.getRoot().getNumberLeaves()); - + s1.releaseSlot(); s2.releaseSlot(); s3.releaseSlot(); s4.releaseSlot(); - + assertEquals(2, scheduler.getNumberOfAvailableSlots()); - + assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfSlots()); assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForGroup(jid1)); assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForGroup(jid2)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java index 25498c4e0ef0c..d78f551f07ff6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java @@ -35,7 +35,6 @@ import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getDummyTask; import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getRandomInstance; import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertex; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -122,17 +121,17 @@ public void testScheduleImmediately() { assertEquals(5, scheduler.getNumberOfAvailableSlots()); // schedule something into all slots - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask())); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask())); - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask())); - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask())); - SimpleSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask())); + SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false).get(); + SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false).get(); + SimpleSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false).get(); + SimpleSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false).get(); + SimpleSlot s5 = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false).get(); // the slots should all be different assertTrue(areAllDistinct(s1, s2, s3, s4, s5)); try { - scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask())); + scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false); fail("Scheduler accepted scheduling request without available resource."); } catch (NoResourceAvailableException e) { @@ -145,8 +144,8 @@ public void testScheduleImmediately() { assertEquals(2, scheduler.getNumberOfAvailableSlots()); // now we can schedule some more slots - SimpleSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask())); - SimpleSlot s7 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask())); + SimpleSlot s6 = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false).get(); + SimpleSlot s7 = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false).get(); assertTrue(areAllDistinct(s1, s2, s3, s4, s5, s6, s7)); @@ -245,7 +244,7 @@ public void run() { disposeThread.start(); for (int i = 0; i < NUM_TASKS_TO_SCHEDULE; i++) { - SlotAllocationFuture future = scheduler.scheduleQueued(new ScheduledUnit(getDummyTask())); + SlotAllocationFuture future = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), true); future.setFutureAction(action); allAllocatedSlots.add(future); } @@ -287,11 +286,11 @@ public void testScheduleWithDyingInstances() { scheduler.newInstanceAvailable(i3); List slots = new ArrayList(); - slots.add(scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask()))); - slots.add(scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask()))); - slots.add(scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask()))); - slots.add(scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask()))); - slots.add(scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask()))); + slots.add(scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false).get()); + slots.add(scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false).get()); + slots.add(scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false).get()); + slots.add(scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false).get()); + slots.add(scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false).get()); i2.markDead(); @@ -312,7 +311,7 @@ public void testScheduleWithDyingInstances() { // cannot get another slot, since all instances are dead try { - scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask())); + scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false).get(); fail("Scheduler served a slot from a dead instance"); } catch (NoResourceAvailableException e) { @@ -347,7 +346,7 @@ public void testSchedulingLocation() { scheduler.newInstanceAvailable(i3); // schedule something on an arbitrary instance - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(new Instance[0]))); + SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(new Instance[0])), false).get(); // figure out how we use the location hints Instance first = (Instance) s1.getOwner(); @@ -355,28 +354,28 @@ public void testSchedulingLocation() { Instance third = first == i3 ? i2 : i3; // something that needs to go to the first instance again - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(s1.getTaskManagerLocation()))); + SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(s1.getTaskManagerLocation())), false).get(); assertEquals(first, s2.getOwner()); // first or second --> second, because first is full - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(first, second))); + SimpleSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(first, second)), false).get(); assertEquals(second, s3.getOwner()); // first or third --> third (because first is full) - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(first, third))); - SimpleSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(first, third))); + SimpleSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), false).get(); + SimpleSlot s5 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), false).get(); assertEquals(third, s4.getOwner()); assertEquals(third, s5.getOwner()); // first or third --> second, because all others are full - SimpleSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(first, third))); + SimpleSlot s6 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), false).get(); assertEquals(second, s6.getOwner()); // release something on the first and second instance s2.releaseSlot(); s6.releaseSlot(); - SimpleSlot s7 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(first, third))); + SimpleSlot s7 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), false).get(); assertEquals(first, s7.getOwner()); assertEquals(1, scheduler.getNumberOfUnconstrainedAssignments()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java index fd0523b985932..c4121f625fe06 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java @@ -62,10 +62,10 @@ public void scheduleSingleVertexType() { scheduler.newInstanceAvailable(i2); // schedule 4 tasks from the first vertex group - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 8), sharingGroup)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 1, 8), sharingGroup)); - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 2, 8), sharingGroup)); - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 3, 8), sharingGroup)); + SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 8), sharingGroup), false).get(); + SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 8), sharingGroup), false).get(); + SimpleSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 8), sharingGroup), false).get(); + SimpleSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 8), sharingGroup), false).get(); assertNotNull(s1); assertNotNull(s2); @@ -76,7 +76,7 @@ public void scheduleSingleVertexType() { // we cannot schedule another task from the first vertex group try { - scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 4, 8), sharingGroup)); + scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 4, 8), sharingGroup), false).get(); fail("Scheduler accepted too many tasks at the same time"); } catch (NoResourceAvailableException e) { @@ -90,7 +90,7 @@ public void scheduleSingleVertexType() { s3.releaseSlot(); // allocate another slot from that group - SimpleSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 4, 8), sharingGroup)); + SimpleSlot s5 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 4, 8), sharingGroup), false).get(); assertNotNull(s5); // release all old slots @@ -98,9 +98,9 @@ public void scheduleSingleVertexType() { s2.releaseSlot(); s4.releaseSlot(); - SimpleSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 5, 8), sharingGroup)); - SimpleSlot s7 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 6, 8), sharingGroup)); - SimpleSlot s8 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 7, 8), sharingGroup)); + SimpleSlot s6 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 5, 8), sharingGroup), false).get(); + SimpleSlot s7 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 6, 8), sharingGroup), false).get(); + SimpleSlot s8 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 7, 8), sharingGroup), false).get(); assertNotNull(s6); assertNotNull(s7); @@ -135,7 +135,7 @@ public void scheduleSingleVertexType() { } @Test - public void scheduleImmediatelyWithSharing() { + public void allocateSlotWithSharing() { try { JobVertexID jid1 = new JobVertexID(); JobVertexID jid2 = new JobVertexID(); @@ -147,10 +147,10 @@ public void scheduleImmediatelyWithSharing() { scheduler.newInstanceAvailable(getRandomInstance(2)); // schedule 4 tasks from the first vertex group - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 5), sharingGroup)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 1, 5), sharingGroup)); - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 2, 5), sharingGroup)); - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 3, 5), sharingGroup)); + SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 5), sharingGroup), false).get(); + SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 5), sharingGroup), false).get(); + SimpleSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 5), sharingGroup), false).get(); + SimpleSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 5), sharingGroup), false).get(); assertNotNull(s1); assertNotNull(s2); @@ -161,7 +161,7 @@ public void scheduleImmediatelyWithSharing() { // we cannot schedule another task from the first vertex group try { - scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 4, 5), sharingGroup)); + scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 4, 5), sharingGroup), false).get(); fail("Scheduler accepted too many tasks at the same time"); } catch (NoResourceAvailableException e) { @@ -172,10 +172,10 @@ public void scheduleImmediatelyWithSharing() { } // schedule some tasks from the second ID group - SimpleSlot s1_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 5), sharingGroup)); - SimpleSlot s2_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 1, 5), sharingGroup)); - SimpleSlot s3_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 2, 5), sharingGroup)); - SimpleSlot s4_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 3, 5), sharingGroup)); + SimpleSlot s1_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 5), sharingGroup), false).get(); + SimpleSlot s2_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 5), sharingGroup), false).get(); + SimpleSlot s3_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 5), sharingGroup), false).get(); + SimpleSlot s4_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 3, 5), sharingGroup), false).get(); assertNotNull(s1_2); assertNotNull(s2_2); @@ -184,7 +184,7 @@ public void scheduleImmediatelyWithSharing() { // we cannot schedule another task from the second vertex group try { - scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 4, 5), sharingGroup)); + scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 4, 5), sharingGroup), false).get(); fail("Scheduler accepted too many tasks at the same time"); } catch (NoResourceAvailableException e) { @@ -205,7 +205,7 @@ public void scheduleImmediatelyWithSharing() { // we can still not schedule anything from the second group of vertices try { - scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 4, 5), sharingGroup)); + scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 4, 5), sharingGroup), false).get(); fail("Scheduler accepted too many tasks at the same time"); } catch (NoResourceAvailableException e) { @@ -216,7 +216,7 @@ public void scheduleImmediatelyWithSharing() { } // we can schedule something from the first vertex group - SimpleSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 4, 5), sharingGroup)); + SimpleSlot s5 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 4, 5), sharingGroup), false).get(); assertNotNull(s5); assertEquals(4, sharingGroup.getTaskAssignment().getNumberOfSlots()); @@ -226,7 +226,7 @@ public void scheduleImmediatelyWithSharing() { // now we release a slot from the second vertex group and schedule another task from that group s2_2.releaseSlot(); - SimpleSlot s5_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 4, 5), sharingGroup)); + SimpleSlot s5_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 4, 5), sharingGroup), false).get(); assertNotNull(s5_2); // release all slots @@ -255,7 +255,7 @@ public void scheduleImmediatelyWithSharing() { } @Test - public void scheduleImmediatelyWithIntermediateTotallyEmptySharingGroup() { + public void allocateSlotWithIntermediateTotallyEmptySharingGroup() { try { JobVertexID jid1 = new JobVertexID(); JobVertexID jid2 = new JobVertexID(); @@ -267,10 +267,10 @@ public void scheduleImmediatelyWithIntermediateTotallyEmptySharingGroup() { scheduler.newInstanceAvailable(getRandomInstance(2)); // schedule 4 tasks from the first vertex group - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 4), sharingGroup)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 1, 4), sharingGroup)); - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup)); - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 3, 4), sharingGroup)); + SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 4), sharingGroup), false).get(); + SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 4), sharingGroup), false).get(); + SimpleSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup), false).get(); + SimpleSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 4), sharingGroup), false).get(); assertEquals(4, sharingGroup.getTaskAssignment().getNumberOfSlots()); assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForGroup(jid1)); @@ -286,10 +286,10 @@ public void scheduleImmediatelyWithIntermediateTotallyEmptySharingGroup() { assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForGroup(jid2)); // schedule some tasks from the second ID group - SimpleSlot s1_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 4), sharingGroup)); - SimpleSlot s2_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 1, 4), sharingGroup)); - SimpleSlot s3_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 2, 4), sharingGroup)); - SimpleSlot s4_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 3, 4), sharingGroup)); + SimpleSlot s1_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 4), sharingGroup), false).get(); + SimpleSlot s2_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 4), sharingGroup), false).get(); + SimpleSlot s3_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 4), sharingGroup), false).get(); + SimpleSlot s4_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 3, 4), sharingGroup), false).get(); assertEquals(4, sharingGroup.getTaskAssignment().getNumberOfSlots()); assertEquals(4, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForGroup(jid1)); @@ -319,7 +319,7 @@ public void scheduleImmediatelyWithIntermediateTotallyEmptySharingGroup() { } @Test - public void scheduleImmediatelyWithTemprarilyEmptyVertexGroup() { + public void allocateSlotWithTemprarilyEmptyVertexGroup() { try { JobVertexID jid1 = new JobVertexID(); JobVertexID jid2 = new JobVertexID(); @@ -332,10 +332,10 @@ public void scheduleImmediatelyWithTemprarilyEmptyVertexGroup() { scheduler.newInstanceAvailable(getRandomInstance(2)); // schedule 4 tasks from the first vertex group - SimpleSlot s1_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 4), sharingGroup)); - SimpleSlot s2_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 1, 4), sharingGroup)); - SimpleSlot s3_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup)); - SimpleSlot s4_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 3, 4), sharingGroup)); + SimpleSlot s1_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 4), sharingGroup), false).get(); + SimpleSlot s2_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 4), sharingGroup), false).get(); + SimpleSlot s3_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup), false).get(); + SimpleSlot s4_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 4), sharingGroup), false).get(); assertNotNull(s1_1); assertNotNull(s2_1); @@ -345,10 +345,10 @@ public void scheduleImmediatelyWithTemprarilyEmptyVertexGroup() { assertTrue(areAllDistinct(s1_1, s2_1, s3_1, s4_1)); // schedule 4 tasks from the second vertex group - SimpleSlot s1_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 7), sharingGroup)); - SimpleSlot s2_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 1, 7), sharingGroup)); - SimpleSlot s3_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 2, 7), sharingGroup)); - SimpleSlot s4_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 3, 7), sharingGroup)); + SimpleSlot s1_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 7), sharingGroup), false).get(); + SimpleSlot s2_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 7), sharingGroup), false).get(); + SimpleSlot s3_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 7), sharingGroup), false).get(); + SimpleSlot s4_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 3, 7), sharingGroup), false).get(); assertNotNull(s1_2); assertNotNull(s2_2); @@ -358,10 +358,10 @@ public void scheduleImmediatelyWithTemprarilyEmptyVertexGroup() { assertTrue(areAllDistinct(s1_2, s2_2, s3_2, s4_2)); // schedule 4 tasks from the third vertex group - SimpleSlot s1_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 0, 4), sharingGroup)); - SimpleSlot s2_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 1, 4), sharingGroup)); - SimpleSlot s3_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 2, 4), sharingGroup)); - SimpleSlot s4_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 3, 4), sharingGroup)); + SimpleSlot s1_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 0, 4), sharingGroup), false).get(); + SimpleSlot s2_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 1, 4), sharingGroup), false).get(); + SimpleSlot s3_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 2, 4), sharingGroup), false).get(); + SimpleSlot s4_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 3, 4), sharingGroup), false).get(); assertNotNull(s1_3); assertNotNull(s2_3); @@ -373,7 +373,7 @@ public void scheduleImmediatelyWithTemprarilyEmptyVertexGroup() { // we cannot schedule another task from the second vertex group try { - scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 4, 5), sharingGroup)); + scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 4, 5), sharingGroup), false).get(); fail("Scheduler accepted too many tasks at the same time"); } catch (NoResourceAvailableException e) { @@ -389,9 +389,9 @@ public void scheduleImmediatelyWithTemprarilyEmptyVertexGroup() { s3_2.releaseSlot(); s4_2.releaseSlot(); - SimpleSlot s5_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 5, 7), sharingGroup)); - SimpleSlot s6_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 6, 7), sharingGroup)); - SimpleSlot s7_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 7, 7), sharingGroup)); + SimpleSlot s5_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 5, 7), sharingGroup), false).get(); + SimpleSlot s6_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 6, 7), sharingGroup), false).get(); + SimpleSlot s7_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 7, 7), sharingGroup), false).get(); assertNotNull(s5_2); assertNotNull(s6_2); @@ -430,7 +430,7 @@ public void scheduleImmediatelyWithTemprarilyEmptyVertexGroup() { } @Test - public void scheduleImmediatelyWithTemprarilyEmptyVertexGroup2() { + public void allocateSlotWithTemporarilyEmptyVertexGroup2() { try { JobVertexID jid1 = new JobVertexID(); JobVertexID jid2 = new JobVertexID(); @@ -442,9 +442,9 @@ public void scheduleImmediatelyWithTemprarilyEmptyVertexGroup2() { scheduler.newInstanceAvailable(getRandomInstance(2)); // schedule 1 tasks from the first vertex group and 2 from the second - SimpleSlot s1_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 2), sharingGroup)); - SimpleSlot s2_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 2), sharingGroup)); - SimpleSlot s2_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 1, 2), sharingGroup)); + SimpleSlot s1_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 2), sharingGroup), false).get(); + SimpleSlot s2_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 2), sharingGroup), false).get(); + SimpleSlot s2_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 2), sharingGroup), false).get(); assertNotNull(s1_1); assertNotNull(s2_1); @@ -460,7 +460,7 @@ public void scheduleImmediatelyWithTemprarilyEmptyVertexGroup2() { // this should free one slot so we can allocate one non-shared - SimpleSlot sx = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 0, 1))); + SimpleSlot sx = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 0, 1)), false).get(); assertNotNull(sx); assertEquals(1, sharingGroup.getTaskAssignment().getNumberOfSlots()); @@ -495,28 +495,28 @@ public void scheduleMixedSharingAndNonSharing() { scheduler.newInstanceAvailable(getRandomInstance(2)); // schedule some individual vertices - SimpleSlot sA1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jidA, 0, 2))); - SimpleSlot sA2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jidA, 1, 2))); + SimpleSlot sA1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidA, 0, 2)), false).get(); + SimpleSlot sA2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidA, 1, 2)), false).get(); assertNotNull(sA1); assertNotNull(sA2); // schedule some vertices in the sharing group - SimpleSlot s1_0 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 4), sharingGroup)); - SimpleSlot s1_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 1, 4), sharingGroup)); - SimpleSlot s2_0 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 4), sharingGroup)); - SimpleSlot s2_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 1, 4), sharingGroup)); + SimpleSlot s1_0 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 4), sharingGroup), false).get(); + SimpleSlot s1_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 4), sharingGroup), false).get(); + SimpleSlot s2_0 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 4), sharingGroup), false).get(); + SimpleSlot s2_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 4), sharingGroup), false).get(); assertNotNull(s1_0); assertNotNull(s1_1); assertNotNull(s2_0); assertNotNull(s2_1); // schedule another isolated vertex - SimpleSlot sB1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jidB, 1, 3))); + SimpleSlot sB1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidB, 1, 3)), false).get(); assertNotNull(sB1); // should not be able to schedule more vertices try { - scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup)); + scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup), false).get(); fail("Scheduler accepted too many tasks at the same time"); } catch (NoResourceAvailableException e) { @@ -527,7 +527,7 @@ public void scheduleMixedSharingAndNonSharing() { } try { - scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 2, 4), sharingGroup)); + scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 4), sharingGroup), false).get(); fail("Scheduler accepted too many tasks at the same time"); } catch (NoResourceAvailableException e) { @@ -538,7 +538,7 @@ public void scheduleMixedSharingAndNonSharing() { } try { - scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jidB, 0, 3))); + scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidB, 0, 3)), false); fail("Scheduler accepted too many tasks at the same time"); } catch (NoResourceAvailableException e) { @@ -549,7 +549,7 @@ public void scheduleMixedSharingAndNonSharing() { } try { - scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jidC, 0, 1))); + scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidC, 0, 1)), false); fail("Scheduler accepted too many tasks at the same time"); } catch (NoResourceAvailableException e) { @@ -562,8 +562,8 @@ public void scheduleMixedSharingAndNonSharing() { // release some isolated task and check that the sharing group may grow sA1.releaseSlot(); - SimpleSlot s1_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup)); - SimpleSlot s2_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 3, 4), sharingGroup)); + SimpleSlot s1_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup), false).get(); + SimpleSlot s2_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 3, 4), sharingGroup), false).get(); assertNotNull(s1_2); assertNotNull(s2_2); @@ -575,19 +575,19 @@ public void scheduleMixedSharingAndNonSharing() { assertEquals(1, scheduler.getNumberOfAvailableSlots()); // schedule one more no-shared task - SimpleSlot sB0 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jidB, 0, 3))); + SimpleSlot sB0 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidB, 0, 3)), false).get(); assertNotNull(sB0); // release the last of the original shared slots and allocate one more non-shared slot s2_1.releaseSlot(); - SimpleSlot sB2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jidB, 2, 3))); + SimpleSlot sB2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidB, 2, 3)), false).get(); assertNotNull(sB2); // release on non-shared and add some shared slots sA2.releaseSlot(); - SimpleSlot s1_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 3, 4), sharingGroup)); - SimpleSlot s2_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 2, 4), sharingGroup)); + SimpleSlot s1_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 4), sharingGroup), false).get(); + SimpleSlot s2_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 4), sharingGroup), false).get(); assertNotNull(s1_3); assertNotNull(s2_3); @@ -597,8 +597,8 @@ public void scheduleMixedSharingAndNonSharing() { s1_3.releaseSlot(); s2_3.releaseSlot(); - SimpleSlot sC0 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jidC, 1, 2))); - SimpleSlot sC1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jidC, 0, 2))); + SimpleSlot sC0 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidC, 1, 2)), false).get(); + SimpleSlot sC1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidC, 0, 2)), false).get(); assertNotNull(sC0); assertNotNull(sC1); @@ -646,8 +646,8 @@ public void testLocalizedAssignment1() { // schedule one to each instance - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup)); + SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup), false).get(); + SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup), false).get(); assertNotNull(s1); assertNotNull(s2); @@ -656,8 +656,8 @@ public void testLocalizedAssignment1() { assertEquals(1, i2.getNumberOfAvailableSlots()); // schedule one from the other group to each instance - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup)); - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc2), sharingGroup)); + SimpleSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup), false).get(); + SimpleSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc2), sharingGroup), false).get(); assertNotNull(s3); assertNotNull(s4); @@ -699,8 +699,8 @@ public void testLocalizedAssignment2() { // schedule one to each instance - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc1), sharingGroup)); + SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup), false).get(); + SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc1), sharingGroup), false).get(); assertNotNull(s1); assertNotNull(s2); @@ -709,8 +709,8 @@ public void testLocalizedAssignment2() { assertEquals(2, i2.getNumberOfAvailableSlots()); // schedule one from the other group to each instance - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc2), sharingGroup)); - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc2), sharingGroup)); + SimpleSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc2), sharingGroup), false).get(); + SimpleSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc2), sharingGroup), false).get(); assertNotNull(s3); assertNotNull(s4); @@ -750,14 +750,14 @@ public void testLocalizedAssignment3() { scheduler.newInstanceAvailable(i2); // schedule until the one instance is full - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc1), sharingGroup)); - SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 4, loc1), sharingGroup)); - SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 4, loc1), sharingGroup)); + SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup), false).get(); + SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc1), sharingGroup), false).get(); + SimpleSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 4, loc1), sharingGroup), false).get(); + SimpleSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 4, loc1), sharingGroup), false).get(); // schedule two more with preference of same instance --> need to go to other instance - SimpleSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 3, 4, loc1), sharingGroup)); - SimpleSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 4, 4, loc1), sharingGroup)); + SimpleSlot s5 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 3, 4, loc1), sharingGroup), false).get(); + SimpleSlot s6 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 4, 4, loc1), sharingGroup), false).get(); assertNotNull(s1); assertNotNull(s2); @@ -803,19 +803,19 @@ public void testSequentialAllocateAndRelease() { scheduler.newInstanceAvailable(getRandomInstance(4)); // allocate something from group 1 and 2 interleaved with schedule for group 3 - SimpleSlot slot_1_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 4), sharingGroup)); - SimpleSlot slot_1_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 1, 4), sharingGroup)); + SimpleSlot slot_1_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 4), sharingGroup), false).get(); + SimpleSlot slot_1_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 4), sharingGroup), false).get(); - SimpleSlot slot_2_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 4), sharingGroup)); - SimpleSlot slot_2_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 1, 4), sharingGroup)); + SimpleSlot slot_2_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 4), sharingGroup), false).get(); + SimpleSlot slot_2_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 4), sharingGroup), false).get(); - SimpleSlot slot_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 0, 1), sharingGroup)); + SimpleSlot slot_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 0, 1), sharingGroup), false).get(); - SimpleSlot slot_1_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup)); - SimpleSlot slot_1_4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 3, 4), sharingGroup)); + SimpleSlot slot_1_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup), false).get(); + SimpleSlot slot_1_4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 4), sharingGroup), false).get(); - SimpleSlot slot_2_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 2, 4), sharingGroup)); - SimpleSlot slot_2_4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 3, 4), sharingGroup)); + SimpleSlot slot_2_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 4), sharingGroup), false).get(); + SimpleSlot slot_2_4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 3, 4), sharingGroup), false).get(); // release groups 1 and 2 @@ -831,10 +831,10 @@ public void testSequentialAllocateAndRelease() { // allocate group 4 - SimpleSlot slot_4_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 0, 4), sharingGroup)); - SimpleSlot slot_4_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 1, 4), sharingGroup)); - SimpleSlot slot_4_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 2, 4), sharingGroup)); - SimpleSlot slot_4_4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 3, 4), sharingGroup)); + SimpleSlot slot_4_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 0, 4), sharingGroup), false).get(); + SimpleSlot slot_4_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 1, 4), sharingGroup), false).get(); + SimpleSlot slot_4_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 2, 4), sharingGroup), false).get(); + SimpleSlot slot_4_4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 3, 4), sharingGroup), false).get(); // release groups 3 and 4 @@ -885,7 +885,7 @@ public void testConcurrentAllocateAndRelease() { @Override public void run() { try { - SimpleSlot slot = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, enumerator4.getAndIncrement(), 4), sharingGroup)); + SimpleSlot slot = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, enumerator4.getAndIncrement(), 4), sharingGroup), false).get(); sleepUninterruptibly(rnd.nextInt(5)); slot.releaseSlot(); @@ -908,7 +908,7 @@ public void run() { public void run() { try { if (flag3.compareAndSet(false, true)) { - SimpleSlot slot = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 0, 1), sharingGroup)); + SimpleSlot slot = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 0, 1), sharingGroup), false).get(); sleepUninterruptibly(5); @@ -937,7 +937,7 @@ public void run() { @Override public void run() { try { - SimpleSlot slot = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, enumerator2.getAndIncrement(), 4), sharingGroup)); + SimpleSlot slot = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, enumerator2.getAndIncrement(), 4), sharingGroup), false).get(); // wait a bit till scheduling the successor sleepUninterruptibly(rnd.nextInt(5)); @@ -964,7 +964,7 @@ public void run() { @Override public void run() { try { - SimpleSlot slot = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, enumerator1.getAndIncrement(), 4), sharingGroup)); + SimpleSlot slot = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, enumerator1.getAndIncrement(), 4), sharingGroup), false).get(); // wait a bit till scheduling the successor sleepUninterruptibly(rnd.nextInt(5)); @@ -1039,27 +1039,27 @@ public void testDopIncreases() { scheduler.newInstanceAvailable(getRandomInstance(4)); // schedule one task for the first and second vertex - SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 1), sharingGroup)); - SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 1), sharingGroup)); + SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 1), sharingGroup), false).get(); + SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 1), sharingGroup), false).get(); assertTrue( s1.getParent() == s2.getParent() ); assertEquals(3, scheduler.getNumberOfAvailableSlots()); - SimpleSlot s3_0 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 0, 5), sharingGroup)); - SimpleSlot s3_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 1, 5), sharingGroup)); - SimpleSlot s4_0 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 0, 4), sharingGroup)); - SimpleSlot s4_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 1, 4), sharingGroup)); + SimpleSlot s3_0 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 0, 5), sharingGroup), false).get(); + SimpleSlot s3_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 1, 5), sharingGroup), false).get(); + SimpleSlot s4_0 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 0, 4), sharingGroup), false).get(); + SimpleSlot s4_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 1, 4), sharingGroup), false).get(); s1.releaseSlot(); s2.releaseSlot(); - SimpleSlot s3_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 2, 5), sharingGroup)); - SimpleSlot s3_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 3, 5), sharingGroup)); - SimpleSlot s4_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 2, 4), sharingGroup)); - SimpleSlot s4_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 3, 4), sharingGroup)); + SimpleSlot s3_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 2, 5), sharingGroup), false).get(); + SimpleSlot s3_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 3, 5), sharingGroup), false).get(); + SimpleSlot s4_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 2, 4), sharingGroup), false).get(); + SimpleSlot s4_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 3, 4), sharingGroup), false).get(); try { - scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 4, 5), sharingGroup)); + scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 4, 5), sharingGroup), false); fail("should throw an exception"); } catch (NoResourceAvailableException e) { From 39449579d582f4bf2768e56e8d58e58044e0346b Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 5 Sep 2016 12:25:28 +0200 Subject: [PATCH 023/299] [hotfix] Delete leftover (superseded) StreamTaskAsyncCheckpointTest There is RocksDBAsyncSnapshotTest which tests async snapshots for the RocksDB state backend. Operators themselves cannot do asynchronous checkpoints right now. --- .../tasks/StreamTaskAsyncCheckpointTest.java | 234 ------------------ 1 file changed, 234 deletions(-) delete mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskAsyncCheckpointTest.java diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskAsyncCheckpointTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskAsyncCheckpointTest.java deleted file mode 100644 index 66bc23730254d..0000000000000 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskAsyncCheckpointTest.java +++ /dev/null @@ -1,234 +0,0 @@ -///* -// * Licensed to the Apache Software Foundation (ASF) under one -// * or more contributor license agreements. See the NOTICE file -// * distributed with this work for additional information -// * regarding copyright ownership. The ASF licenses this file -// * to you under the Apache License, Version 2.0 (the -// * "License"); you may not use this file except in compliance -// * with the License. You may obtain a copy of the License at -// * -// * http://www.apache.org/licenses/LICENSE-2.0 -// * -// * Unless required by applicable law or agreed to in writing, software -// * distributed under the License is distributed on an "AS IS" BASIS, -// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// * See the License for the specific language governing permissions and -// * limitations under the License. -// */ -// -//package org.apache.flink.streaming.runtime.tasks; -// -//import org.apache.flink.api.common.ExecutionConfig; -//import org.apache.flink.api.common.functions.MapFunction; -//import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -//import org.apache.flink.core.testutils.OneShotLatch; -//import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; -//import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; -//import org.apache.flink.streaming.api.graph.StreamConfig; -//import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -//import org.apache.flink.streaming.api.operators.OneInputStreamOperator; -//import org.apache.flink.streaming.api.watermark.Watermark; -//import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -//import org.junit.Test; -//import org.junit.runner.RunWith; -//import org.powermock.core.classloader.annotations.PowerMockIgnore; -//import org.powermock.core.classloader.annotations.PrepareForTest; -//import org.powermock.modules.junit4.PowerMockRunner; -// -//import java.io.IOException; -//import java.lang.reflect.Field; -// -//import static org.junit.Assert.assertEquals; -//import static org.junit.Assert.assertTrue; -// -///** -// * Tests for asynchronous checkpoints. -// */ -//@RunWith(PowerMockRunner.class) -//@PrepareForTest(ResultPartitionWriter.class) -//@PowerMockIgnore({"javax.management.*", "com.sun.jndi.*"}) -//@SuppressWarnings("serial") -//public class StreamTaskAsyncCheckpointTest { -// -// /** -// * This ensures that asynchronous state handles are actually materialized asynchonously. -// * -// *

We use latches to block at various stages and see if the code still continues through -// * the parts that are not asynchronous. If the checkpoint is not done asynchronously the -// * test will simply lock forever. -// * @throws Exception -// */ -// @Test -// public void testAsyncCheckpoints() throws Exception { -// final OneShotLatch delayCheckpointLatch = new OneShotLatch(); -// final OneShotLatch ensureCheckpointLatch = new OneShotLatch(); -// -// final OneInputStreamTask task = new OneInputStreamTask<>(); -// -// final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness<>(task, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); -// -// StreamConfig streamConfig = testHarness.getStreamConfig(); -// -// streamConfig.setStreamOperator(new AsyncCheckpointOperator()); -// -// StreamMockEnvironment mockEnv = new StreamMockEnvironment( -// testHarness.jobConfig, -// testHarness.taskConfig, -// testHarness.memorySize, -// new MockInputSplitProvider(), -// testHarness.bufferSize) { -// -// @Override -// public ExecutionConfig getExecutionConfig() { -// return testHarness.executionConfig; -// } -// -// @Override -// public void acknowledgeCheckpoint(long checkpointId) { -// super.acknowledgeCheckpoint(checkpointId); -// } -// -// @Override -// public void acknowledgeCheckpoint(long checkpointId, StateHandle state) { -// super.acknowledgeCheckpoint(checkpointId, state); -// -// // block on the latch, to verify that triggerCheckpoint returns below, -// // even though the async checkpoint would not finish -// try { -// delayCheckpointLatch.await(); -// } catch (InterruptedException e) { -// e.printStackTrace(); -// } -// -// assertTrue(state instanceof StreamTaskStateList); -// StreamTaskStateList stateList = (StreamTaskStateList) state; -// -// // should be only one state -// StreamTaskState taskState = stateList.getState(this.getUserClassLoader())[0]; -// StateHandle operatorState = taskState.getOperatorState(); -// assertTrue("It must be a TestStateHandle", operatorState instanceof TestStateHandle); -// TestStateHandle testState = (TestStateHandle) operatorState; -// assertEquals(42, testState.checkpointId); -// assertEquals(17, testState.timestamp); -// -// // we now know that the checkpoint went through -// ensureCheckpointLatch.trigger(); -// } -// }; -// -// testHarness.invoke(mockEnv); -// -// // wait for the task to be running -// for (Field field: StreamTask.class.getDeclaredFields()) { -// if (field.getName().equals("isRunning")) { -// field.setAccessible(true); -// while (!field.getBoolean(task)) { -// Thread.sleep(10); -// } -// -// } -// } -// -// task.triggerCheckpoint(42, 17); -// -// // now we allow the checkpoint -// delayCheckpointLatch.trigger(); -// -// // wait for the checkpoint to go through -// ensureCheckpointLatch.await(); -// -// testHarness.endInput(); -// testHarness.waitForTaskCompletion(); -// } -// -// -// // ------------------------------------------------------------------------ -// -// public static class AsyncCheckpointOperator -// extends AbstractStreamOperator -// implements OneInputStreamOperator { -// @Override -// public void processElement(StreamRecord element) throws Exception { -// // we also don't care -// } -// -// @Override -// public void processWatermark(Watermark mark) throws Exception { -// // not interested -// } -// -// -// @Override -// public StreamTaskState snapshotOperatorState(final long checkpointId, final long timestamp) throws Exception { -// StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp); -// -// AsynchronousStateHandle asyncState = -// new DataInputViewAsynchronousStateHandle(checkpointId, timestamp); -// -// taskState.setOperatorState(asyncState); -// -// return taskState; -// } -// -// @Override -// public void restoreState(StreamTaskState taskState) throws Exception { -// super.restoreState(taskState); -// } -// } -// -// private static class DataInputViewAsynchronousStateHandle extends AsynchronousStateHandle { -// -// private final long checkpointId; -// private final long timestamp; -// -// public DataInputViewAsynchronousStateHandle(long checkpointId, long timestamp) { -// this.checkpointId = checkpointId; -// this.timestamp = timestamp; -// } -// -// @Override -// public StateHandle materialize() throws Exception { -// return new TestStateHandle(checkpointId, timestamp); -// } -// -// @Override -// public long getStateSize() { -// return 0; -// } -// -// @Override -// public void close() throws IOException {} -// } -// -// private static class TestStateHandle implements StateHandle { -// -// public final long checkpointId; -// public final long timestamp; -// -// public TestStateHandle(long checkpointId, long timestamp) { -// this.checkpointId = checkpointId; -// this.timestamp = timestamp; -// } -// -// @Override -// public String getState(ClassLoader userCodeClassLoader) throws Exception { -// return null; -// } -// -// @Override -// public void discardState() throws Exception {} -// -// @Override -// public long getStateSize() { -// return 0; -// } -// -// @Override -// public void close() throws IOException {} -// } -// -// public static class DummyMapFunction implements MapFunction { -// @Override -// public T map(T value) { return value; } -// } -//} From 9aca8187af964fabc2d5ada7bea0469d63da4fa7 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Mon, 5 Sep 2016 16:05:50 +0200 Subject: [PATCH 024/299] [FLINK-4073] fix stability of TaskManagerFailure test --- .../yarn/YARNSessionCapacitySchedulerITCase.java | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java index 513a9fc0b9415..886ec327125b0 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java @@ -29,12 +29,13 @@ import org.apache.flink.test.util.TestBaseUtils; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest; +import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.NodeReport; -import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.api.records.NodeState; -import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.security.NMTokenIdentifier; @@ -249,11 +250,16 @@ public void testTaskManagerFailure() { List nodeReports = yc.getNodeReports(NodeState.RUNNING); // we asked for one node with 2 vcores so we expect 2 vcores - int userVcores = 0; + // note that the JobManager may also run on the NodeManager + boolean foundVCoresSetting = false; for (NodeReport rep: nodeReports) { - userVcores += rep.getUsed().getVirtualCores(); + Resource resource = rep.getUsed(); + if (resource != null && resource.getVirtualCores() == 2) { + foundVCoresSetting = true; + break; + } } - Assert.assertEquals(2, userVcores); + Assert.assertTrue(foundVCoresSetting); } catch (Exception e) { Assert.fail("Test failed: " + e.getMessage()); } From b055dbf6d202758a28dc398db8107465749f886d Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Tue, 6 Sep 2016 12:22:04 +0200 Subject: [PATCH 025/299] [FLINK-4073] remove unstable testing code in TaskManagerFailure test Yarn reports null or (1, maxVcores) depending on its internal logic. The test only worked in the past because it summed up the used vcores of the RM and the TM containers. We have checks in place to ensure the vcores config value is passed on to the Flink ResourceManager. --- .../YARNSessionCapacitySchedulerITCase.java | 21 ------------------- 1 file changed, 21 deletions(-) diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java index 886ec327125b0..ddea4ddea0bd0 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java @@ -32,9 +32,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.NodeReport; -import org.apache.hadoop.yarn.api.records.NodeState; -import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.conf.YarnConfiguration; @@ -246,24 +243,6 @@ public void testTaskManagerFailure() { Assert.assertNotNull("Unable to find container with TaskManager", taskManagerContainer); Assert.assertNotNull("Illegal state", nodeManager); - try { - List nodeReports = yc.getNodeReports(NodeState.RUNNING); - - // we asked for one node with 2 vcores so we expect 2 vcores - // note that the JobManager may also run on the NodeManager - boolean foundVCoresSetting = false; - for (NodeReport rep: nodeReports) { - Resource resource = rep.getUsed(); - if (resource != null && resource.getVirtualCores() == 2) { - foundVCoresSetting = true; - break; - } - } - Assert.assertTrue(foundVCoresSetting); - } catch (Exception e) { - Assert.fail("Test failed: " + e.getMessage()); - } - yc.stop(); List toStop = new LinkedList(); From 45a93c35419db6d6c8e0ece22f51a91c9c9e761c Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Sun, 4 Sep 2016 01:37:39 +0800 Subject: [PATCH 026/299] [FLINK-4559][kinesis-connector] Fix AWSUtil.getCredentialsProvider() to not return null Return a DefaultAWSCredentialsProviderChain instead of null when AWS_CREDENTIALS_PROVIDER config is set to "AUTO" This closes #2470 --- .../streaming/connectors/kinesis/util/AWSUtil.java | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/AWSUtil.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/AWSUtil.java index 06f967b9c28aa..81c0b6b1d55ec 100644 --- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/AWSUtil.java +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/AWSUtil.java @@ -22,6 +22,7 @@ import com.amazonaws.auth.AWSCredentials; import com.amazonaws.auth.AWSCredentialsProvider; import com.amazonaws.auth.BasicAWSCredentials; +import com.amazonaws.auth.DefaultAWSCredentialsProviderChain; import com.amazonaws.auth.EnvironmentVariableCredentialsProvider; import com.amazonaws.auth.SystemPropertiesCredentialsProvider; import com.amazonaws.auth.profile.ProfileCredentialsProvider; @@ -50,13 +51,8 @@ public static AmazonKinesisClient createKinesisClient(Properties configProps) { awsClientConfig.setUserAgent("Apache Flink " + EnvironmentInformation.getVersion() + " (" + EnvironmentInformation.getRevisionInformation().commitId + ") Kinesis Connector"); - AmazonKinesisClient client; - if (AWSUtil.getCredentialsProvider(configProps) != null) { - client = new AmazonKinesisClient( - AWSUtil.getCredentialsProvider(configProps).getCredentials(), awsClientConfig); - } else { - client = new AmazonKinesisClient(awsClientConfig); - } + AmazonKinesisClient client = new AmazonKinesisClient( + AWSUtil.getCredentialsProvider(configProps).getCredentials(), awsClientConfig); client.setRegion(Region.getRegion(Regions.fromName(configProps.getProperty(AWSConfigConstants.AWS_REGION)))); if (configProps.containsKey(AWSConfigConstants.AWS_ENDPOINT)) { @@ -94,7 +90,7 @@ public static AWSCredentialsProvider getCredentialsProvider(final Properties con : new ProfileCredentialsProvider(profileConfigPath, profileName); break; case AUTO: - credentialsProvider = null; + credentialsProvider = new DefaultAWSCredentialsProviderChain(); break; default: case BASIC: From 78b72afcdd12970957ba2ef8f7dd11afb8c4e785 Mon Sep 17 00:00:00 2001 From: Greg Hogan Date: Mon, 25 Jul 2016 14:05:56 -0400 Subject: [PATCH 027/299] [FLINK-4265] [dataset api] Add a NoOpOperator Adds a NoOpOperator which is unwound in OperatorTranslation.translate. This will be first used by Gelly as a placeholder to support implicit operator reuse. This closes #2294 --- .../api/java/operators/NoOpOperator.java | 51 +++++++++++++++++++ .../java/operators/OperatorTranslation.java | 20 ++++---- 2 files changed, 62 insertions(+), 9 deletions(-) create mode 100644 flink-java/src/main/java/org/apache/flink/api/java/operators/NoOpOperator.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/NoOpOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/NoOpOperator.java new file mode 100644 index 0000000000000..369ab9e13210a --- /dev/null +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/NoOpOperator.java @@ -0,0 +1,51 @@ +/* + * 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.operators; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.util.Preconditions; + +/** + * This operator will be ignored during translation. + * + * @param The type of the data set passed through the operator. + */ +@Internal +public class NoOpOperator extends DataSet { + + private DataSet input; + + public NoOpOperator(DataSet input, TypeInformation resultType) { + super(input.getExecutionEnvironment(), resultType); + + this.input = input; + } + + public DataSet getInput() { + return input; + } + + public void setInput(DataSet input) { + Preconditions.checkNotNull(input); + + this.input = input; + } +} diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java index 74811a3ce54a1..3f44d58e3914b 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java @@ -19,15 +19,8 @@ package org.apache.flink.api.java.operators; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.java.DataSet; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - import org.apache.flink.api.common.InvalidProgramException; +import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.operators.AbstractUdfOperator; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.GenericDataSinkBase; @@ -35,8 +28,14 @@ import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.BulkIterationBase; import org.apache.flink.api.common.operators.base.DeltaIterationBase; +import org.apache.flink.api.java.DataSet; import org.apache.flink.configuration.Configuration; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + @Internal public class OperatorTranslation { @@ -70,7 +69,10 @@ private GenericDataSinkBase translate(DataSink sink) { private Operator translate(DataSet dataSet) { - + while (dataSet instanceof NoOpOperator) { + dataSet = ((NoOpOperator) dataSet).getInput(); + } + // check if we have already translated that data set (operation or source) Operator previous = (Operator) this.translated.get(dataSet); if (previous != null) { From b583b9e71663ac154236ebb53c982c2e0c3ade21 Mon Sep 17 00:00:00 2001 From: f7753 Date: Mon, 22 Aug 2016 21:07:55 +0800 Subject: [PATCH 028/299] [FLINK-4436] Unclosed DataOutputBuffer in Utils#setTokensFor() This closes #2402 --- .../main/java/org/apache/flink/yarn/Utils.java | 15 ++++++++------- .../flink/yarn/YarnApplicationMasterRunner.java | 3 +-- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java index d5bad2f7914ee..1496d61c90e19 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java @@ -155,15 +155,16 @@ public static void setTokensFor(ContainerLaunchContext amContainer, List p LOG.info("Adding user token " + id + " with " + token); credentials.addToken(id, token); } - DataOutputBuffer dob = new DataOutputBuffer(); - credentials.writeTokenStorageToStream(dob); + try (DataOutputBuffer dob = new DataOutputBuffer()) { + credentials.writeTokenStorageToStream(dob); - if(LOG.isDebugEnabled()) { - LOG.debug("Wrote tokens. Credentials buffer length: " + dob.getLength()); - } + if(LOG.isDebugEnabled()) { + LOG.debug("Wrote tokens. Credentials buffer length: " + dob.getLength()); + } - ByteBuffer securityTokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength()); - amContainer.setTokens(securityTokens); + ByteBuffer securityTokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength()); + amContainer.setTokens(securityTokens); + } } /** diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java index 4637b97ac80f0..745334406a709 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java @@ -599,10 +599,9 @@ public static ContainerLaunchContext createTaskManagerContext( ctx.setEnvironment(containerEnv); - try { + try (DataOutputBuffer dob = new DataOutputBuffer()) { UserGroupInformation user = UserGroupInformation.getCurrentUser(); Credentials credentials = user.getCredentials(); - DataOutputBuffer dob = new DataOutputBuffer(); credentials.writeTokenStorageToStream(dob); ByteBuffer securityTokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength()); ctx.setTokens(securityTokens); From 55eaa93963aaa01e4b253bcc7eaa5a8a9b28356f Mon Sep 17 00:00:00 2001 From: Alexander Pivovarov Date: Wed, 7 Sep 2016 14:11:06 -0700 Subject: [PATCH 029/299] [FLINK-4595] Close FileOutputStream in ParameterTool This closes #2478 --- .../java/org/apache/flink/api/java/utils/ParameterTool.java | 5 ++++- .../org/apache/flink/api/java/utils/ParameterToolTest.java | 5 ++++- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java b/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java index 46c73879fb3df..38ae6df8ee73e 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java @@ -31,6 +31,7 @@ import java.io.FileNotFoundException; import java.io.FileOutputStream; import java.io.IOException; +import java.io.OutputStream; import java.io.Serializable; import java.util.Arrays; import java.util.HashMap; @@ -523,7 +524,9 @@ public void createPropertiesFile(String pathToFile, boolean overwrite) throws IO } Properties defaultProps = new Properties(); defaultProps.putAll(this.defaultData); - defaultProps.store(new FileOutputStream(file), "Default file created by Flink's ParameterUtil.createPropertiesFile()"); + try (final OutputStream out = new FileOutputStream(file)) { + defaultProps.store(out, "Default file created by Flink's ParameterUtil.createPropertiesFile()"); + } } @Override diff --git a/flink-java/src/test/java/org/apache/flink/api/java/utils/ParameterToolTest.java b/flink-java/src/test/java/org/apache/flink/api/java/utils/ParameterToolTest.java index a070473fe0597..09a778130b5d6 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/utils/ParameterToolTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/utils/ParameterToolTest.java @@ -29,6 +29,7 @@ import java.io.FileInputStream; import java.io.FileOutputStream; import java.io.IOException; +import java.io.OutputStream; import java.util.Map; import java.util.Properties; @@ -118,7 +119,9 @@ public void testFromPropertiesFile() throws IOException { Properties props = new Properties(); props.setProperty("input", "myInput"); props.setProperty("expectedCount", "15"); - props.store(new FileOutputStream(propertiesFile), "Test properties"); + try (final OutputStream out = new FileOutputStream(propertiesFile)) { + props.store(out, "Test properties"); + } ParameterTool parameter = ParameterTool.fromPropertiesFile(propertiesFile.getAbsolutePath()); Assert.assertEquals(2, parameter.getNumberOfParameters()); validate(parameter); From 0e1d38cd6ec09eb9609bec0997b1d0459a1f618e Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 31 Aug 2016 17:58:09 +0200 Subject: [PATCH 030/299] [FLINK-4458] Replace ForkableFlinkMiniCluster by LocalFlinkMiniCluster Rename _configuration to originalConfiguration Remove testing classes from main scope in flink-runtime Previously, the ForkableFlinkMiniCluster which resided in flink-test-utils required these files to be in the main scope of flink-runtime. With the removal of the ForkableFlinkMiniCluster, these classes are now no longer needed and can be moved back to the test scope. This closes #2450. --- .../avro/AvroExternalJarProgramITCase.java | 7 +- .../contrib/streaming/CollectITCase.java | 4 +- .../DegreesWithExceptionITCase.java | 6 +- .../ReduceOnEdgesWithExceptionITCase.java | 6 +- .../ReduceOnNeighborsWithExceptionITCase.java | 6 +- .../apache/flink/ml/util/FlinkTestBase.scala | 11 +- .../FlinkResourceManager.java | 13 +- .../flink/runtime/jobmanager/JobManager.scala | 45 ++- .../messages/TaskManagerMessages.scala | 26 ++ .../minicluster/FlinkMiniCluster.scala | 73 ++-- .../minicluster/LocalFlinkMiniCluster.scala | 235 +++++++++--- .../runtime/taskmanager/TaskManager.scala | 130 +++++-- ...LeaderElectionRetrievalTestingCluster.java | 3 +- .../testutils/TestingResourceManager.java | 0 .../runtime/testingUtils/TestingCluster.scala | 322 ++++++++++------- .../testingUtils/TestingJobManager.scala | 37 +- .../testingUtils/TestingJobManagerLike.scala | 2 +- .../TestingJobManagerMessages.scala | 3 +- .../testingUtils/TestingMemoryArchivist.scala | 0 .../testingUtils/TestingMessages.scala | 0 .../testingUtils/TestingTaskManager.scala | 0 .../testingUtils/TestingTaskManagerLike.scala | 14 - .../TestingTaskManagerMessages.scala | 14 +- .../flink/api/scala/ScalaShellITCase.scala | 7 +- .../cassandra/CassandraConnectorITCase.java | 6 +- .../kafka/KafkaShortRetentionTestBase.java | 6 +- .../connectors/kafka/KafkaTestBase.java | 6 +- .../manualtests/ManualExactlyOnceTest.java | 4 +- ...alExactlyOnceWithStreamReshardingTest.java | 4 +- ...alaStreamingMultipleProgramsTestBase.scala | 5 +- .../flink-test-utils/pom.xml | 149 -------- .../StreamingMultipleProgramsTestBase.java | 4 +- .../streaming/util/TestStreamEnvironment.java | 8 +- .../flink/test/util/AbstractTestBase.java | 3 +- .../test/util/MultipleProgramsTestBase.java | 3 +- .../apache/flink/test/util/TestBaseUtils.java | 31 +- .../flink/test/util/TestEnvironment.java | 7 +- .../test/util/ForkableFlinkMiniCluster.scala | 335 ------------------ .../accumulators/AccumulatorErrorITCase.java | 6 +- .../accumulators/AccumulatorLiveITCase.java | 1 - .../test/cancelling/CancelingTestBase.java | 7 +- ...EventTimeAllWindowCheckpointingITCase.java | 6 +- .../EventTimeWindowCheckpointingITCase.java | 6 +- .../test/checkpointing/RescalingITCase.java | 6 +- .../test/checkpointing/SavepointITCase.java | 19 +- .../StreamCheckpointNotifierITCase.java | 6 +- .../StreamFaultToleranceTestBase.java | 6 +- .../WindowCheckpointingITCase.java | 6 +- .../test/classloading/ClassLoaderITCase.java | 7 +- .../clients/examples/JobRetrievalITCase.java | 5 +- .../JobSubmissionFailsITCase.java | 6 +- .../CustomDistributionITCase.java | 4 +- .../RemoteEnvironmentITCase.java | 7 +- .../test/misc/AutoParallelismITCase.java | 6 +- .../test/misc/CustomSerializationITCase.java | 6 +- .../test/misc/MiscellaneousIssuesITCase.java | 6 +- ...ccessAfterNetworkBuffersFailureITCase.java | 6 +- .../test/query/QueryableStateITCase.java | 6 +- .../test/recovery/FastFailuresITCase.java | 4 +- ...mpleRecoveryFailureRateStrategyITBase.java | 6 +- ...coveryFixedDelayRestartStrategyITBase.java | 6 +- .../recovery/SimpleRecoveryITCaseBase.java | 4 +- .../TaskManagerFailureRecoveryITCase.java | 6 +- .../test/runtime/IPv6HostnamesITCase.java | 6 +- .../ZooKeeperLeaderElectionITCase.java | 56 +-- .../streaming/runtime/TimestampITCase.java | 6 +- .../flink/test/web/WebFrontendITCase.java | 6 +- .../jobmanager/JobManagerFailsITCase.scala | 8 +- .../taskmanager/TaskManagerFailsITCase.scala | 12 +- flink-yarn-tests/pom.xml | 8 + .../org/apache/flink/yarn/YarnTestBase.java | 1 - tools/maven/scalastyle-config.xml | 2 +- 72 files changed, 832 insertions(+), 962 deletions(-) rename flink-runtime/src/{main => test}/java/org/apache/flink/runtime/testutils/TestingResourceManager.java (100%) rename flink-runtime/src/{main => test}/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala (70%) rename flink-runtime/src/{main => test}/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala (99%) rename flink-runtime/src/{main => test}/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala (98%) rename flink-runtime/src/{main => test}/scala/org/apache/flink/runtime/testingUtils/TestingMemoryArchivist.scala (100%) rename flink-runtime/src/{main => test}/scala/org/apache/flink/runtime/testingUtils/TestingMessages.scala (100%) rename flink-runtime/src/{main => test}/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala (100%) rename flink-runtime/src/{main => test}/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerLike.scala (94%) rename flink-runtime/src/{main => test}/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerMessages.scala (89%) delete mode 100644 flink-test-utils-parent/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala diff --git a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java b/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java index 29a7e586d5d22..1030ff82032ae 100644 --- a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java +++ b/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java @@ -25,12 +25,11 @@ import org.apache.flink.client.program.StandaloneClusterClient; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.junit.Assert; import org.junit.Test; - public class AvroExternalJarProgramITCase { private static final String JAR_FILE = "maven-test-jar.jar"; @@ -40,12 +39,12 @@ public class AvroExternalJarProgramITCase { @Test public void testExternalProgram() { - ForkableFlinkMiniCluster testMiniCluster = null; + LocalFlinkMiniCluster testMiniCluster = null; try { Configuration config = new Configuration(); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 4); - testMiniCluster = new ForkableFlinkMiniCluster(config, false); + testMiniCluster = new LocalFlinkMiniCluster(config, false); testMiniCluster.start(); String jarFile = JAR_FILE; diff --git a/flink-contrib/flink-streaming-contrib/src/test/java/org/apache/flink/contrib/streaming/CollectITCase.java b/flink-contrib/flink-streaming-contrib/src/test/java/org/apache/flink/contrib/streaming/CollectITCase.java index 10ea85c042ffe..d6916219ee3b7 100644 --- a/flink-contrib/flink-streaming-contrib/src/test/java/org/apache/flink/contrib/streaming/CollectITCase.java +++ b/flink-contrib/flink-streaming-contrib/src/test/java/org/apache/flink/contrib/streaming/CollectITCase.java @@ -19,9 +19,9 @@ package org.apache.flink.contrib.streaming; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.junit.Test; @@ -36,7 +36,7 @@ public class CollectITCase { @Test public void testCollect() throws Exception { - final ForkableFlinkMiniCluster cluster = new ForkableFlinkMiniCluster(new Configuration(), false); + final LocalFlinkMiniCluster cluster = new LocalFlinkMiniCluster(new Configuration(), false); try { cluster.start(); diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java index 551a97b8bbcf8..02eea077e8428 100644 --- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java +++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java @@ -26,7 +26,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.graph.Graph; import org.apache.flink.graph.test.TestGraphUtils; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.types.LongValue; import org.junit.AfterClass; @@ -39,7 +39,7 @@ public class DegreesWithExceptionITCase { private static final int PARALLELISM = 4; - private static ForkableFlinkMiniCluster cluster; + private static LocalFlinkMiniCluster cluster; @BeforeClass @@ -47,7 +47,7 @@ public static void setupCluster() { try { Configuration config = new Configuration(); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM); - cluster = new ForkableFlinkMiniCluster(config, false); + cluster = new LocalFlinkMiniCluster(config, false); cluster.start(); } catch (Exception e) { diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java index 56a0a5909d39c..666f7efc6a216 100644 --- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java +++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java @@ -30,7 +30,7 @@ import org.apache.flink.graph.Graph; import org.apache.flink.graph.Vertex; import org.apache.flink.graph.test.TestGraphUtils; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.util.Collector; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -42,7 +42,7 @@ public class ReduceOnEdgesWithExceptionITCase { private static final int PARALLELISM = 4; - private static ForkableFlinkMiniCluster cluster; + private static LocalFlinkMiniCluster cluster; @BeforeClass @@ -50,7 +50,7 @@ public static void setupCluster() { try { Configuration config = new Configuration(); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM); - cluster = new ForkableFlinkMiniCluster(config, false); + cluster = new LocalFlinkMiniCluster(config, false); cluster.start(); } catch (Exception e) { diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java index 7458e0821a8fd..0bbdc84517c66 100644 --- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java +++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java @@ -31,7 +31,7 @@ import org.apache.flink.graph.ReduceNeighborsFunction; import org.apache.flink.graph.Vertex; import org.apache.flink.graph.test.TestGraphUtils; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.util.Collector; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -43,7 +43,7 @@ public class ReduceOnNeighborsWithExceptionITCase { private static final int PARALLELISM = 4; - private static ForkableFlinkMiniCluster cluster; + private static LocalFlinkMiniCluster cluster; @BeforeClass @@ -51,7 +51,7 @@ public static void setupCluster() { try { Configuration config = new Configuration(); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM); - cluster = new ForkableFlinkMiniCluster(config, false); + cluster = new LocalFlinkMiniCluster(config, false); cluster.start(); } catch (Exception e) { diff --git a/flink-libraries/flink-ml/src/test/scala/org/apache/flink/ml/util/FlinkTestBase.scala b/flink-libraries/flink-ml/src/test/scala/org/apache/flink/ml/util/FlinkTestBase.scala index fb98f24cf0a16..6353d6a61e44c 100644 --- a/flink-libraries/flink-ml/src/test/scala/org/apache/flink/ml/util/FlinkTestBase.scala +++ b/flink-libraries/flink-ml/src/test/scala/org/apache/flink/ml/util/FlinkTestBase.scala @@ -18,14 +18,15 @@ package org.apache.flink.ml.util -import org.apache.flink.test.util.{ForkableFlinkMiniCluster, TestBaseUtils, TestEnvironment} +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster +import org.apache.flink.test.util.{TestBaseUtils, TestEnvironment} import org.scalatest.{BeforeAndAfter, Suite} -/** Mixin to start and stop a ForkableFlinkMiniCluster automatically for Scala based tests. +/** Mixin to start and stop a LocalFlinkMiniCluster automatically for Scala based tests. * Additionally a TestEnvironment with the started cluster is created and set as the default * [[org.apache.flink.api.java.ExecutionEnvironment]]. * - * This mixin starts a ForkableFlinkMiniCluster with one TaskManager and a number of slots given + * This mixin starts a LocalFlinkMiniCluster with one TaskManager and a number of slots given * by parallelism. This value can be overridden in a sub class in order to start the cluster * with a different number of slots. * @@ -37,7 +38,7 @@ import org.scalatest.{BeforeAndAfter, Suite} * @example * {{{ * def testSomething: Unit = { - * // Obtain TestEnvironment with started ForkableFlinkMiniCluster + * // Obtain TestEnvironment with started LocalFlinkMiniCluster * val env = ExecutionEnvironment.getExecutionEnvironment * * env.fromCollection(...) @@ -50,7 +51,7 @@ import org.scalatest.{BeforeAndAfter, Suite} trait FlinkTestBase extends BeforeAndAfter { that: Suite => - var cluster: Option[ForkableFlinkMiniCluster] = None + var cluster: Option[LocalFlinkMiniCluster] = None val parallelism = 4 before { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/FlinkResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/FlinkResourceManager.java index 95be084b4c335..7ea286d4439ba 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/FlinkResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/FlinkResourceManager.java @@ -767,8 +767,19 @@ public static ActorRef startResourceManagerActors( Class> resourceManagerClass, String resourceManagerActorName) { - Props resourceMasterProps = Props.create(resourceManagerClass, configuration, leaderRetriever); + Props resourceMasterProps = getResourceManagerProps( + resourceManagerClass, + configuration, + leaderRetriever); return actorSystem.actorOf(resourceMasterProps, resourceManagerActorName); } + + public static Props getResourceManagerProps( + Class resourceManagerClass, + Configuration configuration, + LeaderRetrievalService leaderRetrievalService) { + + return Props.create(resourceManagerClass, configuration, leaderRetrievalService); + } } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index 88af604504f29..f67be0ed370b2 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -49,7 +49,7 @@ import org.apache.flink.runtime.clusterframework.standalone.StandaloneResourceMa import org.apache.flink.runtime.clusterframework.types.ResourceID import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory -import org.apache.flink.runtime.executiongraph.{StatusListenerMessenger, ExecutionGraph, ExecutionJobVertex} +import org.apache.flink.runtime.executiongraph.{ExecutionGraph, ExecutionJobVertex, StatusListenerMessenger} import org.apache.flink.runtime.instance.{AkkaActorGateway, InstanceManager} import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator import org.apache.flink.runtime.jobgraph.{JobGraph, JobStatus, JobVertexID} @@ -2721,7 +2721,7 @@ object JobManager { configuration, None) - val archiveProps = Props(archiveClass, archiveCount) + val archiveProps = getArchiveProps(archiveClass, archiveCount) // start the archiver with the given name, or without (avoid name conflicts) val archive: ActorRef = archiveActorName match { @@ -2729,7 +2729,7 @@ object JobManager { case None => actorSystem.actorOf(archiveProps) } - val jobManagerProps = Props( + val jobManagerProps = getJobManagerProps( jobManagerClass, configuration, executorService, @@ -2754,6 +2754,45 @@ object JobManager { (jobManager, archive) } + def getArchiveProps(archiveClass: Class[_ <: MemoryArchivist], archiveCount: Int): Props = { + Props(archiveClass, archiveCount) + } + + def getJobManagerProps( + jobManagerClass: Class[_ <: JobManager], + configuration: Configuration, + executorService: ExecutorService, + instanceManager: InstanceManager, + scheduler: FlinkScheduler, + libraryCacheManager: BlobLibraryCacheManager, + archive: ActorRef, + restartStrategyFactory: RestartStrategyFactory, + timeout: FiniteDuration, + leaderElectionService: LeaderElectionService, + submittedJobGraphStore: SubmittedJobGraphStore, + checkpointRecoveryFactory: CheckpointRecoveryFactory, + savepointStore: SavepointStore, + jobRecoveryTimeout: FiniteDuration, + metricsRegistry: Option[FlinkMetricRegistry]): Props = { + + Props( + jobManagerClass, + configuration, + executorService, + instanceManager, + scheduler, + libraryCacheManager, + archive, + restartStrategyFactory, + timeout, + leaderElectionService, + submittedJobGraphStore, + checkpointRecoveryFactory, + savepointStore, + jobRecoveryTimeout, + metricsRegistry) + } + // -------------------------------------------------------------------------- // Resolving the JobManager endpoint // -------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala index 2d99245aac6e2..b43301510c518 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala @@ -130,6 +130,16 @@ object TaskManagerMessages { */ case class RequestTaskManagerLog(requestType : LogTypeRequest) + /** Requests the number of active connections at the ConnectionManager */ + case object RequestNumActiveConnections + + case class ResponseNumActiveConnections(number: Int) + + /** Requests the number of broadcast variables with references */ + case object RequestBroadcastVariablesWithReferences + + case class ResponseBroadcastVariablesWithReferences(number: Int) + // -------------------------------------------------------------------------- // Utility getters for case objects to simplify access from Java @@ -166,4 +176,20 @@ object TaskManagerMessages { def getRequestTaskManagerStdout(): AnyRef = { RequestTaskManagerLog(StdOutFileRequest) } + + /** + * Accessor for the case object instance, to simplify Java interoperability. + * @return The RequestBroadcastVariablesWithReferences case object instance. + */ + def getRequestBroadcastVariablesWithReferences(): RequestBroadcastVariablesWithReferences.type = { + RequestBroadcastVariablesWithReferences + } + + /** + * Accessor for the case object instance, to simplify Java interoperability. + * @return The RequestNumActiveConnections case object instance. + */ + def getRequestNumActiveConnections(): RequestNumActiveConnections.type = { + RequestNumActiveConnections + } } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala index a547d25f7001b..0178bd38c1b0b 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala @@ -69,7 +69,7 @@ abstract class FlinkMiniCluster( ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, InetAddress.getByName("localhost").getHostAddress()) - val configuration = generateConfiguration(userConfiguration) + protected val originalConfiguration = generateConfiguration(userConfiguration) /** Future to the [[ActorGateway]] of the current leader */ var leaderGateway: Promise[ActorGateway] = Promise() @@ -79,16 +79,16 @@ abstract class FlinkMiniCluster( /** Future lock */ val futureLock = new Object() - + implicit val executionContext = ExecutionContext.global - implicit val timeout = AkkaUtils.getTimeout(configuration) + implicit val timeout = AkkaUtils.getTimeout(originalConfiguration) - val haMode = HighAvailabilityMode.fromConfig(configuration) + val haMode = HighAvailabilityMode.fromConfig(originalConfiguration) val numJobManagers = getNumberOfJobManagers - var numTaskManagers = configuration.getInteger( + var numTaskManagers = originalConfiguration.getInteger( ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, ConfigConstants.DEFAULT_LOCAL_NUMBER_TASK_MANAGER) @@ -105,6 +105,22 @@ abstract class FlinkMiniCluster( private var isRunning = false + def configuration: Configuration = { + if (originalConfiguration.getInteger( + ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, + ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT) == 0) { + val leaderConfiguration = new Configuration(originalConfiguration) + + val leaderPort = getLeaderRPCPort + + leaderConfiguration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, leaderPort) + + leaderConfiguration + } else { + originalConfiguration + } + } + // -------------------------------------------------------------------------- // Abstract Methods // -------------------------------------------------------------------------- @@ -125,7 +141,7 @@ abstract class FlinkMiniCluster( if(haMode == HighAvailabilityMode.NONE) { 1 } else { - configuration.getInteger( + originalConfiguration.getInteger( ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, ConfigConstants.DEFAULT_LOCAL_NUMBER_JOB_MANAGER ) @@ -136,7 +152,7 @@ abstract class FlinkMiniCluster( if(haMode == HighAvailabilityMode.NONE) { 1 } else { - configuration.getInteger( + originalConfiguration.getInteger( ConfigConstants.LOCAL_NUMBER_RESOURCE_MANAGER, ConfigConstants.DEFAULT_LOCAL_NUMBER_RESOURCE_MANAGER ) @@ -177,40 +193,55 @@ abstract class FlinkMiniCluster( Await.result(indexFuture, timeout) } + def getLeaderRPCPort: Int = { + val index = getLeaderIndex(timeout) + + jobManagerActorSystems match { + case Some(jmActorSystems) => + AkkaUtils.getAddress(jmActorSystems(index)).port match { + case Some(p) => p + case None => -1 + } + + case None => throw new Exception("The JobManager of the LocalFlinkMiniCluster has not been " + + "started properly.") + } + } + def getResourceManagerAkkaConfig(index: Int): Config = { if (useSingleActorSystem) { - AkkaUtils.getAkkaConfig(configuration, None) + AkkaUtils.getAkkaConfig(originalConfiguration, None) } else { - val port = configuration.getInteger(ConfigConstants.RESOURCE_MANAGER_IPC_PORT_KEY, - ConfigConstants.DEFAULT_RESOURCE_MANAGER_IPC_PORT) + val port = originalConfiguration.getInteger(ConfigConstants.RESOURCE_MANAGER_IPC_PORT_KEY, + ConfigConstants.DEFAULT_RESOURCE_MANAGER_IPC_PORT) val resolvedPort = if(port != 0) port + index else port - AkkaUtils.getAkkaConfig(configuration, Some((hostname, resolvedPort))) + AkkaUtils.getAkkaConfig(originalConfiguration, Some((hostname, resolvedPort))) } } def getJobManagerAkkaConfig(index: Int): Config = { if (useSingleActorSystem) { - AkkaUtils.getAkkaConfig(configuration, None) + AkkaUtils.getAkkaConfig(originalConfiguration, None) } else { - val port = configuration.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, - ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT) + val port = originalConfiguration.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, + ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT) val resolvedPort = if(port != 0) port + index else port - AkkaUtils.getAkkaConfig(configuration, Some((hostname, resolvedPort))) + AkkaUtils.getAkkaConfig(originalConfiguration, Some((hostname, resolvedPort))) } } def getTaskManagerAkkaConfig(index: Int): Config = { - val port = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, - ConfigConstants.DEFAULT_TASK_MANAGER_IPC_PORT) + val port = originalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, + ConfigConstants.DEFAULT_TASK_MANAGER_IPC_PORT) val resolvedPort = if(port != 0) port + index else port - AkkaUtils.getAkkaConfig(configuration, Some((hostname, resolvedPort))) + AkkaUtils.getAkkaConfig(originalConfiguration, Some((hostname, resolvedPort))) } /** @@ -257,7 +288,7 @@ abstract class FlinkMiniCluster( "The FlinkMiniCluster has not been started yet.") } } else { - JobClient.startJobClientActorSystem(configuration) + JobClient.startJobClientActorSystem(originalConfiguration) } } @@ -320,7 +351,7 @@ abstract class FlinkMiniCluster( val jobManagerAkkaURL = AkkaUtils.getAkkaURL(jmActorSystems(0), jmActors(0)) - webMonitor = startWebServer(configuration, jmActorSystems(0), jobManagerAkkaURL) + webMonitor = startWebServer(originalConfiguration, jmActorSystems(0), jobManagerAkkaURL) if(waitForTaskManagerRegistration) { waitForTaskManagersToBeRegistered() @@ -528,7 +559,7 @@ abstract class FlinkMiniCluster( new StandaloneLeaderRetrievalService( AkkaUtils.getAkkaURL(jmActorSystems(0), jmActors(0))) } else { - ZooKeeperUtils.createLeaderRetrievalService(configuration) + ZooKeeperUtils.createLeaderRetrievalService(originalConfiguration) } case _ => throw new Exception("The FlinkMiniCluster has not been started properly.") diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala index d30c0470deb42..cac5d91716cb0 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala @@ -18,23 +18,36 @@ package org.apache.flink.runtime.minicluster -import akka.actor.{ActorRef, ActorSystem} -import org.apache.flink.api.common.JobID +import java.util.concurrent.ExecutorService +import akka.actor.{ActorRef, ActorSystem, Props} +import org.apache.flink.api.common.JobID import org.apache.flink.api.common.io.FileOutputFormat import org.apache.flink.configuration.{ConfigConstants, Configuration} -import org.apache.flink.runtime.akka.AkkaUtils +import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory +import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore import org.apache.flink.runtime.clusterframework.FlinkResourceManager import org.apache.flink.runtime.clusterframework.standalone.StandaloneResourceManager -import org.apache.flink.runtime.clusterframework.types.ResourceID +import org.apache.flink.runtime.clusterframework.types.{ResourceID, ResourceIDRetrievable} +import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager +import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory +import org.apache.flink.runtime.instance.InstanceManager +import org.apache.flink.runtime.io.disk.iomanager.IOManager +import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.io.network.netty.NettyConfig -import org.apache.flink.runtime.jobmanager.{MemoryArchivist, JobManager} +import org.apache.flink.runtime.jobmanager.scheduler.Scheduler +import org.apache.flink.runtime.jobmanager.{JobManager, MemoryArchivist, SubmittedJobGraphStore} +import org.apache.flink.runtime.leaderelection.LeaderElectionService +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService +import org.apache.flink.runtime.memory.MemoryManager import org.apache.flink.runtime.messages.JobManagerMessages -import org.apache.flink.runtime.messages.JobManagerMessages.{CancellationFailure, CancellationResponse, StoppingFailure, StoppingResponse, RunningJobsStatus, RunningJobs} -import org.apache.flink.runtime.taskmanager.TaskManager +import org.apache.flink.runtime.messages.JobManagerMessages.{RunningJobsStatus, StoppingFailure, StoppingResponse} +import org.apache.flink.runtime.metrics.MetricRegistry +import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerConfiguration, TaskManagerLocation} import org.apache.flink.runtime.util.EnvironmentInformation import scala.concurrent.Await +import scala.concurrent.duration.FiniteDuration /** * Local Flink mini cluster which executes all [[TaskManager]]s and the [[JobManager]] in the same @@ -65,8 +78,25 @@ class LocalFlinkMiniCluster( config } + //------------------------------------------------------------------------------------------------ + // Actor classes + //------------------------------------------------------------------------------------------------ + + val jobManagerClass: Class[_ <: JobManager] = classOf[JobManager] + + val taskManagerClass: Class[_ <: TaskManager] = classOf[TaskManager] + + val memoryArchivistClass: Class[_ <: MemoryArchivist] = classOf[MemoryArchivist] + + val resourceManagerClass: Class[_ <: FlinkResourceManager[_ <: ResourceIDRetrievable]] = + classOf[StandaloneResourceManager] + + //------------------------------------------------------------------------------------------------ + // Start methods for the distributed components + //------------------------------------------------------------------------------------------------ + override def startJobManager(index: Int, system: ActorSystem): ActorRef = { - val config = configuration.clone() + val config = originalConfiguration.clone() val jobManagerName = getJobManagerName(index) val archiveName = getArchiveName(index) @@ -79,19 +109,48 @@ class LocalFlinkMiniCluster( config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerPort + index) } - val (jobManager, _) = JobManager.startJobManagerActors( - config, - system, - Some(jobManagerName), - Some(archiveName), - classOf[JobManager], - classOf[MemoryArchivist]) - - jobManager + val (executorService, + instanceManager, + scheduler, + libraryCacheManager, + restartStrategyFactory, + timeout, + archiveCount, + leaderElectionService, + submittedJobGraphStore, + checkpointRecoveryFactory, + savepointStore, + jobRecoveryTimeout, + metricsRegistry) = JobManager.createJobManagerComponents(config, createLeaderElectionService()) + + val archive = system.actorOf( + getArchiveProps( + memoryArchivistClass, + archiveCount), + archiveName) + + system.actorOf( + getJobManagerProps( + jobManagerClass, + config, + executorService, + instanceManager, + scheduler, + libraryCacheManager, + archive, + restartStrategyFactory, + timeout, + leaderElectionService, + submittedJobGraphStore, + checkpointRecoveryFactory, + savepointStore, + jobRecoveryTimeout, + metricsRegistry), + jobManagerName) } override def startResourceManager(index: Int, system: ActorSystem): ActorRef = { - val config = configuration.clone() + val config = originalConfiguration.clone() val resourceManagerName = getResourceManagerName(index) @@ -103,18 +162,16 @@ class LocalFlinkMiniCluster( config.setInteger(ConfigConstants.RESOURCE_MANAGER_IPC_PORT_KEY, resourceManagerPort + index) } - val resourceManager = FlinkResourceManager.startResourceManagerActors( + val resourceManagerProps = getResourceManagerProps( + resourceManagerClass, config, - system, - createLeaderRetrievalService(), - classOf[StandaloneResourceManager], - resourceManagerName) + createLeaderRetrievalService()) - resourceManager + system.actorOf(resourceManagerProps, resourceManagerName) } override def startTaskManager(index: Int, system: ActorSystem): ActorRef = { - val config = configuration.clone() + val config = originalConfiguration.clone() val rpcPort = config.getInteger( ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, @@ -138,32 +195,115 @@ class LocalFlinkMiniCluster( } else { TaskManager.TASK_MANAGER_NAME } - - TaskManager.startTaskManagerComponentsAndActor( + + val resourceID = ResourceID.generate() // generate random resource id + + val (taskManagerConfig, + taskManagerLocation, + memoryManager, + ioManager, + network, + leaderRetrievalService) = TaskManager.createTaskManagerComponents( config, - ResourceID.generate(), // generate random resource id - system, + resourceID, hostname, // network interface to bind to - Some(taskManagerActorName), // actor name - Some(createLeaderRetrievalService()), // job manager leader retrieval service localExecution, // start network stack? - classOf[TaskManager]) + Some(createLeaderRetrievalService())) + + val props = getTaskManagerProps( + taskManagerClass, + taskManagerConfig, + resourceID, + taskManagerLocation, + memoryManager, + ioManager, + network, + leaderRetrievalService) + + system.actorOf(props, taskManagerActorName) } - def getLeaderRPCPort: Int = { - val index = getLeaderIndex(timeout) + //------------------------------------------------------------------------------------------------ + // Props for the distributed components + //------------------------------------------------------------------------------------------------ - jobManagerActorSystems match { - case Some(jmActorSystems) => - AkkaUtils.getAddress(jmActorSystems(index)).port match { - case Some(p) => p - case None => -1 - } + def getArchiveProps(archiveClass: Class[_ <: MemoryArchivist], archiveCount: Int): Props = { + JobManager.getArchiveProps(archiveClass, archiveCount) + } - case None => throw new Exception("The JobManager of the LocalFlinkMiniCluster has not been " + - "started properly.") - } + def getJobManagerProps( + jobManagerClass: Class[_ <: JobManager], + configuration: Configuration, + executorService: ExecutorService, + instanceManager: InstanceManager, + scheduler: Scheduler, + libraryCacheManager: BlobLibraryCacheManager, + archive: ActorRef, + restartStrategyFactory: RestartStrategyFactory, + timeout: FiniteDuration, + leaderElectionService: LeaderElectionService, + submittedJobGraphStore: SubmittedJobGraphStore, + checkpointRecoveryFactory: CheckpointRecoveryFactory, + savepointStore: SavepointStore, + jobRecoveryTimeout: FiniteDuration, + metricsRegistry: Option[MetricRegistry]): Props = { + + JobManager.getJobManagerProps( + jobManagerClass, + configuration, + executorService, + instanceManager, + scheduler, + libraryCacheManager, + archive, + restartStrategyFactory, + timeout, + leaderElectionService, + submittedJobGraphStore, + checkpointRecoveryFactory, + savepointStore, + jobRecoveryTimeout, + metricsRegistry) + } + + def getTaskManagerProps( + taskManagerClass: Class[_ <: TaskManager], + taskManagerConfig: TaskManagerConfiguration, + resourceID: ResourceID, + taskManagerLocation: TaskManagerLocation, + memoryManager: MemoryManager, + ioManager: IOManager, + networkEnvironment: NetworkEnvironment, + leaderRetrievalService: LeaderRetrievalService): Props = { + + TaskManager.getTaskManagerProps( + taskManagerClass, + taskManagerConfig, + resourceID, + taskManagerLocation, + memoryManager, + ioManager, + networkEnvironment, + leaderRetrievalService) + } + + def getResourceManagerProps( + resourceManagerClass: Class[_ <: FlinkResourceManager[_ <: ResourceIDRetrievable]], + configuration: Configuration, + leaderRetrievalService: LeaderRetrievalService): Props = { + + FlinkResourceManager.getResourceManagerProps( + resourceManagerClass, + configuration, + leaderRetrievalService) + } + + //------------------------------------------------------------------------------------------------ + // Helper methods + //------------------------------------------------------------------------------------------------ + def createLeaderElectionService(): Option[LeaderElectionService] = { + None } def initializeIOFormatClasses(configuration: Configuration): Unit = { @@ -186,7 +326,7 @@ class LocalFlinkMiniCluster( val bufferSize: Int = config.getInteger( ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY, ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE) - + val bufferMem: Long = config.getLong( ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS) * bufferSize.toLong @@ -218,6 +358,7 @@ class LocalFlinkMiniCluster( val config: Configuration = new Configuration() config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, hostname) + config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 0) config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, ConfigConstants.DEFAULT_LOCAL_NUMBER_TASK_MANAGER) @@ -252,11 +393,11 @@ class LocalFlinkMiniCluster( JobManager.ARCHIVE_NAME } } - + // -------------------------------------------------------------------------- // Actions on running jobs // -------------------------------------------------------------------------- - + def currentlyRunningJobs: Iterable[JobID] = { val leader = getLeaderGateway(timeout) val future = leader.ask(JobManagerMessages.RequestRunningJobsStatus, timeout) @@ -269,7 +410,7 @@ class LocalFlinkMiniCluster( currentlyRunningJobs.foreach(list.add) list } - + def stopJob(id: JobID) : Unit = { val leader = getLeaderGateway(timeout) val response = leader.ask(new JobManagerMessages.StopJob(id), timeout) diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index 84750a389af8a..de85f30f35590 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -354,6 +354,21 @@ class TaskManager( case None => sender() ! new IOException("BlobService not available. Cannot upload TaskManager logs.") } + + case RequestBroadcastVariablesWithReferences => + sender ! decorateMessage( + ResponseBroadcastVariablesWithReferences( + bcVarManager.getNumberOfVariablesWithReferences) + ) + + case RequestNumActiveConnections => + val numActive = if (!network.isShutdown) { + network.getConnectionManager.getNumberOfActiveConnections + } else { + 0 + } + + sender ! decorateMessage(ResponseNumActiveConnections(numActive)) } /** @@ -1781,6 +1796,7 @@ object TaskManager { } /** + * Starts the task manager actor. * * @param configuration The configuration for the TaskManager. * @param resourceID The id of the resource which the task manager will run on. @@ -1817,11 +1833,75 @@ object TaskManager { taskManagerClass: Class[_ <: TaskManager]) : ActorRef = { - val (taskManagerConfig : TaskManagerConfiguration, - netConfig: NetworkEnvironmentConfiguration, - taskManagerAddress: InetSocketAddress, - memType: MemoryType - ) = parseTaskManagerConfiguration( + val (taskManagerConfig, + connectionInfo, + memoryManager, + ioManager, + network, + leaderRetrievalService) = createTaskManagerComponents( + configuration, + resourceID, + taskManagerHostname, + localTaskManagerCommunication, + leaderRetrievalServiceOption) + + // create the actor properties (which define the actor constructor parameters) + val tmProps = getTaskManagerProps( + taskManagerClass, + taskManagerConfig, + resourceID, + connectionInfo, + memoryManager, + ioManager, + network, + leaderRetrievalService) + + taskManagerActorName match { + case Some(actorName) => actorSystem.actorOf(tmProps, actorName) + case None => actorSystem.actorOf(tmProps) + } + } + + def getTaskManagerProps( + taskManagerClass: Class[_ <: TaskManager], + taskManagerConfig: TaskManagerConfiguration, + resourceID: ResourceID, + taskManagerLocation: TaskManagerLocation, + memoryManager: MemoryManager, + ioManager: IOManager, + networkEnvironment: NetworkEnvironment, + leaderRetrievalService: LeaderRetrievalService + ): Props = { + Props( + taskManagerClass, + taskManagerConfig, + resourceID, + taskManagerLocation, + memoryManager, + ioManager, + networkEnvironment, + taskManagerConfig.numberOfSlots, + leaderRetrievalService) + } + + def createTaskManagerComponents( + configuration: Configuration, + resourceID: ResourceID, + taskManagerHostname: String, + localTaskManagerCommunication: Boolean, + leaderRetrievalServiceOption: Option[LeaderRetrievalService]): + (TaskManagerConfiguration, + TaskManagerLocation, + MemoryManager, + IOManager, + NetworkEnvironment, + LeaderRetrievalService) = { + + val (taskManagerConfig : TaskManagerConfiguration, + netConfig: NetworkEnvironmentConfiguration, + taskManagerAddress: InetSocketAddress, + memType: MemoryType + ) = parseTaskManagerConfiguration( configuration, taskManagerHostname, localTaskManagerCommunication) @@ -1895,10 +1975,10 @@ object TaskManager { // check if a value has been configured val configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L) checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory, - ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, - "MemoryManager needs at least one MB of memory. " + - "If you leave this config parameter empty, the system automatically " + - "pick a fraction of the available memory.") + ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, + "MemoryManager needs at least one MB of memory. " + + "If you leave this config parameter empty, the system automatically " + + "pick a fraction of the available memory.") val preAllocateMemory = configuration.getBoolean( @@ -1910,7 +1990,7 @@ object TaskManager { LOG.info(s"Using $configuredMemory MB for managed memory.") } else { LOG.info(s"Limiting managed memory to $configuredMemory MB, " + - s"memory will be allocated lazily.") + s"memory will be allocated lazily.") } configuredMemory << 20 // megabytes to bytes } @@ -1928,10 +2008,10 @@ object TaskManager { if (preAllocateMemory) { LOG.info(s"Using $fraction of the currently free heap space for managed " + - s"heap memory (${relativeMemSize >> 20} MB).") + s"heap memory (${relativeMemSize >> 20} MB).") } else { LOG.info(s"Limiting managed memory to $fraction of the currently free heap space " + - s"(${relativeMemSize >> 20} MB), memory will be allocated lazily.") + s"(${relativeMemSize >> 20} MB), memory will be allocated lazily.") } relativeMemSize @@ -1944,10 +2024,10 @@ object TaskManager { if (preAllocateMemory) { LOG.info(s"Using $fraction of the maximum memory size for " + - s"managed off-heap memory (${directMemorySize >> 20} MB).") + s"managed off-heap memory (${directMemorySize >> 20} MB).") } else { LOG.info(s"Limiting managed memory to $fraction of the maximum memory size " + - s"(${directMemorySize >> 20} MB), memory will be allocated lazily.") + s"(${directMemorySize >> 20} MB), memory will be allocated lazily.") } directMemorySize @@ -1971,12 +2051,12 @@ object TaskManager { memType match { case MemoryType.HEAP => throw new Exception(s"OutOfMemory error (${e.getMessage()})" + - s" while allocating the TaskManager heap memory ($memorySize bytes).", e) + s" while allocating the TaskManager heap memory ($memorySize bytes).", e) case MemoryType.OFF_HEAP => throw new Exception(s"OutOfMemory error (${e.getMessage()})" + - s" while allocating the TaskManager off-heap memory ($memorySize bytes). " + - s"Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e) + s" while allocating the TaskManager off-heap memory ($memorySize bytes). " + + s"Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e) case _ => throw e } @@ -1990,22 +2070,12 @@ object TaskManager { case None => LeaderRetrievalUtils.createLeaderRetrievalService(configuration) } - // create the actor properties (which define the actor constructor parameters) - val tmProps = Props( - taskManagerClass, - taskManagerConfig, - resourceID, + (taskManagerConfig, taskManagerLocation, memoryManager, ioManager, network, - taskManagerConfig.numberOfSlots, leaderRetrievalService) - - taskManagerActorName match { - case Some(actorName) => actorSystem.actorOf(tmProps, actorName) - case None => actorSystem.actorOf(tmProps) - } } @@ -2055,8 +2125,8 @@ object TaskManager { * @param taskManagerHostname The host name under which the TaskManager communicates. * @param localTaskManagerCommunication True, to skip initializing the network stack. * Use only in cases where only one task manager runs. - * @return A tuple (TaskManagerConfiguration, network configuration, - * InstanceConnectionInfo, JobManager actor Akka URL). + * @return A tuple (TaskManagerConfiguration, network configuration, inet socket address, + * memory tyep). */ @throws(classOf[IllegalArgumentException]) def parseTaskManagerConfiguration( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionRetrievalTestingCluster.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionRetrievalTestingCluster.java index e596166452c14..c143fe2408a78 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionRetrievalTestingCluster.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionRetrievalTestingCluster.java @@ -20,7 +20,6 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.testingUtils.TestingCluster; import scala.Option; @@ -86,7 +85,7 @@ public LeaderRetrievalService createLeaderRetrievalService() { @Override public int getNumberOfJobManagers() { - return this.configuration().getInteger( + return this.originalConfiguration().getInteger( ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, ConfigConstants.DEFAULT_LOCAL_NUMBER_JOB_MANAGER); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/testutils/TestingResourceManager.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingResourceManager.java similarity index 100% rename from flink-runtime/src/main/java/org/apache/flink/runtime/testutils/TestingResourceManager.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingResourceManager.java diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala index b4ba40bc0da52..c01a32125a82b 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala @@ -18,22 +18,32 @@ package org.apache.flink.runtime.testingUtils -import java.util.concurrent.TimeoutException +import java.util.concurrent.{ExecutorService, TimeUnit, TimeoutException} import akka.pattern.ask -import akka.actor.{ActorRef, Props, ActorSystem} +import akka.actor.{ActorRef, ActorSystem, Props} +import akka.pattern.Patterns._ import akka.testkit.CallingThreadDispatcher import org.apache.flink.configuration.{ConfigConstants, Configuration} +import org.apache.flink.runtime.akka.AkkaUtils +import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory +import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore import org.apache.flink.runtime.clusterframework.FlinkResourceManager -import org.apache.flink.runtime.clusterframework.types.ResourceID -import org.apache.flink.runtime.executiongraph.restart.RestartStrategy -import org.apache.flink.runtime.jobmanager.JobManager +import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable +import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager +import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory +import org.apache.flink.runtime.instance.InstanceManager +import org.apache.flink.runtime.jobmanager.scheduler.Scheduler +import org.apache.flink.runtime.jobmanager.{JobManager, MemoryArchivist, SubmittedJobGraphStore} import org.apache.flink.runtime.leaderelection.LeaderElectionService -import org.apache.flink.runtime.minicluster.FlinkMiniCluster +import org.apache.flink.runtime.metrics.MetricRegistry +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster import org.apache.flink.runtime.testutils.TestingResourceManager import org.apache.flink.runtime.taskmanager.TaskManager import org.apache.flink.runtime.testingUtils.TestingMessages.Alive +import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages.NotifyWhenRegisteredAtJobManager +import scala.concurrent.duration.FiniteDuration import scala.concurrent.{Await, Future} /** @@ -48,7 +58,7 @@ class TestingCluster( userConfiguration: Configuration, singleActorSystem: Boolean, synchronousDispatcher: Boolean) - extends FlinkMiniCluster( + extends LocalFlinkMiniCluster( userConfiguration, singleActorSystem) { @@ -59,133 +69,54 @@ class TestingCluster( // -------------------------------------------------------------------------- - override def generateConfiguration(userConfig: Configuration): Configuration = { - val cfg = new Configuration() - cfg.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost") - cfg.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 0) - cfg.setInteger(ConfigConstants.RESOURCE_MANAGER_IPC_PORT_KEY, 0) - cfg.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 10) - cfg.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, -1) - - setDefaultCiConfig(cfg) - - cfg.addAll(userConfig) - cfg - } - - override def startJobManager(index: Int, actorSystem: ActorSystem): ActorRef = { - val config = configuration.clone() - - val jobManagerName = if(singleActorSystem) { - JobManager.JOB_MANAGER_NAME + "_" + (index + 1) - } else { - JobManager.JOB_MANAGER_NAME - } - - val archiveName = if(singleActorSystem) { - JobManager.ARCHIVE_NAME + "_" + (index + 1) - } else { - JobManager.ARCHIVE_NAME - } - - val jobManagerPort = config.getInteger( - ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, - ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT) - - if(jobManagerPort > 0) { - config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerPort + index) - } - - val (executionContext, - instanceManager, - scheduler, - libraryCacheManager, - restartStrategyFactory, - timeout, - archiveCount, - leaderElectionService, - submittedJobsGraphs, - checkpointRecoveryFactory, - savepointStore, - jobRecoveryTimeout, - metricRegistry) = JobManager.createJobManagerComponents( - config, - createLeaderElectionService()) - - val testArchiveProps = Props(new TestingMemoryArchivist(archiveCount)) - val archive = actorSystem.actorOf(testArchiveProps, archiveName) - - val jobManagerProps = Props( - new TestingJobManager( - configuration, - executionContext, - instanceManager, - scheduler, - libraryCacheManager, - archive, - restartStrategyFactory, - timeout, - leaderElectionService, - submittedJobsGraphs, - checkpointRecoveryFactory, - savepointStore, - jobRecoveryTimeout, - metricRegistry)) - - val dispatcherJobManagerProps = if (synchronousDispatcher) { - // disable asynchronous futures (e.g. accumulator update in Heartbeat) - jobManagerProps.withDispatcher(CallingThreadDispatcher.Id) - } else { - jobManagerProps - } - - actorSystem.actorOf(dispatcherJobManagerProps, jobManagerName) - } - - override def startResourceManager(index: Int, system: ActorSystem): ActorRef = { - val config = configuration.clone() - - val resourceManagerName = if(singleActorSystem) { - FlinkResourceManager.RESOURCE_MANAGER_NAME + "_" + (index + 1) + override val jobManagerClass: Class[_ <: JobManager] = classOf[TestingJobManager] + + override val resourceManagerClass: Class[_ <: FlinkResourceManager[_ <: ResourceIDRetrievable]] = + classOf[TestingResourceManager] + + override val taskManagerClass: Class[_ <: TaskManager] = classOf[TestingTaskManager] + + override val memoryArchivistClass: Class[_ <: MemoryArchivist] = classOf[TestingMemoryArchivist] + + override def getJobManagerProps( + jobManagerClass: Class[_ <: JobManager], + configuration: Configuration, + executorService: ExecutorService, + instanceManager: InstanceManager, + scheduler: Scheduler, + libraryCacheManager: BlobLibraryCacheManager, + archive: ActorRef, + restartStrategyFactory: RestartStrategyFactory, + timeout: FiniteDuration, + leaderElectionService: LeaderElectionService, + submittedJobGraphStore: SubmittedJobGraphStore, + checkpointRecoveryFactory: CheckpointRecoveryFactory, + savepointStore: SavepointStore, + jobRecoveryTimeout: FiniteDuration, + metricsRegistry: Option[MetricRegistry]): Props = { + + val props = super.getJobManagerProps( + jobManagerClass, + configuration, + executorService, + instanceManager, + scheduler, + libraryCacheManager, + archive, + restartStrategyFactory, + timeout, + leaderElectionService, + submittedJobGraphStore, + checkpointRecoveryFactory, + savepointStore, + jobRecoveryTimeout, + metricsRegistry) + + if (synchronousDispatcher) { + props.withDispatcher(CallingThreadDispatcher.Id) } else { - FlinkResourceManager.RESOURCE_MANAGER_NAME + props } - - val resourceManagerPort = config.getInteger( - ConfigConstants.RESOURCE_MANAGER_IPC_PORT_KEY, - ConfigConstants.DEFAULT_RESOURCE_MANAGER_IPC_PORT) - - if(resourceManagerPort > 0) { - config.setInteger(ConfigConstants.RESOURCE_MANAGER_IPC_PORT_KEY, resourceManagerPort + index) - } - - val testResourceManagerProps = Props( - new TestingResourceManager( - config, - createLeaderRetrievalService() - )) - - system.actorOf(testResourceManagerProps, resourceManagerName) - } - - override def startTaskManager(index: Int, system: ActorSystem) = { - - val tmActorName = TaskManager.TASK_MANAGER_NAME + "_" + (index + 1) - - TaskManager.startTaskManagerComponentsAndActor( - configuration, - ResourceID.generate(), - system, - hostname, - Some(tmActorName), - Some(createLeaderRetrievalService()), - numTaskManagers == 1, - classOf[TestingTaskManager]) - } - - - def createLeaderElectionService(): Option[LeaderElectionService] = { - None } @throws(classOf[TimeoutException]) @@ -228,4 +159,131 @@ class TestingCluster( Await.ready(combinedFuture, timeout) } + + def waitForTaskManagersToBeRegisteredAtJobManager(jobManager: ActorRef): Unit = { + val futures = taskManagerActors.map { + _.map { + tm => (tm ? NotifyWhenRegisteredAtJobManager(jobManager))(timeout) + } + }.getOrElse(Seq()) + + try { + Await.ready(Future.sequence(futures), timeout) + } catch { + case t: TimeoutException => + throw new Exception("Timeout while waiting for TaskManagers to register at " + + s"${jobManager.path}") + } + + } + + def restartLeadingJobManager(): Unit = { + this.synchronized { + (jobManagerActorSystems, jobManagerActors) match { + case (Some(jmActorSystems), Some(jmActors)) => + val leader = getLeaderGateway(AkkaUtils.getTimeout(originalConfiguration)) + val index = getLeaderIndex(AkkaUtils.getTimeout(originalConfiguration)) + + // restart the leading job manager with the same port + val port = getLeaderRPCPort + val oldPort = originalConfiguration.getInteger( + ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, + 0) + + // we have to set the old port in the configuration file because this is used for startup + originalConfiguration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, port) + + clearLeader() + + val stopped = gracefulStop(leader.actor(), TestingCluster.MAX_RESTART_DURATION) + Await.result(stopped, TestingCluster.MAX_RESTART_DURATION) + + if(!singleActorSystem) { + jmActorSystems(index).shutdown() + jmActorSystems(index).awaitTermination() + } + + val newJobManagerActorSystem = if(!singleActorSystem) { + startJobManagerActorSystem(index) + } else { + jmActorSystems.head + } + + // reset the original configuration + originalConfiguration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, oldPort) + + val newJobManagerActor = startJobManager(index, newJobManagerActorSystem) + + jobManagerActors = Some(jmActors.patch(index, Seq(newJobManagerActor), 1)) + jobManagerActorSystems = Some(jmActorSystems.patch( + index, + Seq(newJobManagerActorSystem), + 1)) + + val lrs = createLeaderRetrievalService() + + jobManagerLeaderRetrievalService = Some(lrs) + lrs.start(this) + + case _ => throw new Exception("The JobManager of the TestingCluster have not " + + "been started properly.") + } + } + } + + def restartTaskManager(index: Int): Unit = { + (taskManagerActorSystems, taskManagerActors) match { + case (Some(tmActorSystems), Some(tmActors)) => + val stopped = gracefulStop(tmActors(index), TestingCluster.MAX_RESTART_DURATION) + Await.result(stopped, TestingCluster.MAX_RESTART_DURATION) + + if(!singleActorSystem) { + tmActorSystems(index).shutdown() + tmActorSystems(index).awaitTermination() + } + + val taskManagerActorSystem = if(!singleActorSystem) { + startTaskManagerActorSystem(index) + } else { + tmActorSystems.head + } + + val taskManagerActor = startTaskManager(index, taskManagerActorSystem) + + taskManagerActors = Some(tmActors.patch(index, Seq(taskManagerActor), 1)) + taskManagerActorSystems = Some(tmActorSystems.patch(index, Seq(taskManagerActorSystem), 1)) + + case _ => throw new Exception("The TaskManager of the TestingCluster have not " + + "been started properly.") + } + } + + def addTaskManager(): Unit = { + if (useSingleActorSystem) { + (jobManagerActorSystems, taskManagerActors) match { + case (Some(jmSystems), Some(tmActors)) => + val index = numTaskManagers + taskManagerActors = Some(tmActors :+ startTaskManager(index, jmSystems(0))) + numTaskManagers += 1 + case _ => throw new IllegalStateException("Cluster has not been started properly.") + } + } else { + (taskManagerActorSystems, taskManagerActors) match { + case (Some(tmSystems), Some(tmActors)) => + val index = numTaskManagers + val newTmSystem = startTaskManagerActorSystem(index) + val newTmActor = startTaskManager(index, newTmSystem) + + taskManagerActorSystems = Some(tmSystems :+ newTmSystem) + taskManagerActors = Some(tmActors :+ newTmActor) + + numTaskManagers += 1 + case _ => throw new IllegalStateException("Cluster has not been started properly.") + } + } + } +} + +object TestingCluster { + val MAX_RESTART_DURATION = new FiniteDuration(2, TimeUnit.MINUTES) } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala similarity index 70% rename from flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala rename to flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala index 16331ac7db872..62349dbd4927b 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala @@ -1,25 +1,26 @@ /* -* 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. -*/ + * 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.runtime.testingUtils -import akka.actor.ActorRef +import java.util.concurrent.ExecutorService +import akka.actor.ActorRef import org.apache.flink.configuration.Configuration import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore @@ -34,8 +35,6 @@ import org.apache.flink.runtime.metrics.MetricRegistry import scala.concurrent.duration._ import scala.language.postfixOps -import java.util.concurrent.ExecutorService - /** JobManager implementation extended by testing messages * */ diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala similarity index 99% rename from flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala rename to flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala index 3947b1738260b..5ba2790f66d0e 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala @@ -228,7 +228,7 @@ trait TestingJobManagerLike extends FlinkActor { val flinkAccumulators = graph.getFlinkAccumulators val userAccumulators = graph.aggregateUserAccumulators actors foreach { - actor => actor ! UpdatedAccumulators(jobID, flinkAccumulators, userAccumulators) + actor => actor ! UpdatedAccumulators(jobID, flinkAccumulators, userAccumulators) } case None => } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala similarity index 98% rename from flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala rename to flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala index f1213052d67c9..d07c48f26c52f 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala @@ -24,11 +24,10 @@ import akka.actor.ActorRef import org.apache.flink.api.common.JobID import org.apache.flink.api.common.accumulators.Accumulator import org.apache.flink.runtime.accumulators.AccumulatorRegistry -import org.apache.flink.runtime.checkpoint.CompletedCheckpoint +import org.apache.flink.runtime.checkpoint.savepoint.Savepoint import org.apache.flink.runtime.executiongraph.{ExecutionAttemptID, ExecutionGraph} import org.apache.flink.runtime.instance.ActorGateway import org.apache.flink.runtime.jobgraph.JobStatus -import org.apache.flink.runtime.checkpoint.savepoint.Savepoint object TestingJobManagerMessages { diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingMemoryArchivist.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingMemoryArchivist.scala similarity index 100% rename from flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingMemoryArchivist.scala rename to flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingMemoryArchivist.scala diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingMessages.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingMessages.scala similarity index 100% rename from flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingMessages.scala rename to flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingMessages.scala diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala similarity index 100% rename from flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala rename to flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerLike.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerLike.scala similarity index 94% rename from flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerLike.scala rename to flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerLike.scala index a6963fe6879af..2498dbe52099e 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerLike.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerLike.scala @@ -101,20 +101,6 @@ trait TestingTaskManagerLike extends FlinkActor { unregisteredTasks += executionID - case RequestBroadcastVariablesWithReferences => - sender ! decorateMessage( - ResponseBroadcastVariablesWithReferences( - bcVarManager.getNumberOfVariablesWithReferences) - ) - - case RequestNumActiveConnections => - val numActive = if (!network.isShutdown) { - network.getConnectionManager.getNumberOfActiveConnections - } else { - 0 - } - sender ! decorateMessage(ResponseNumActiveConnections(numActive)) - case NotifyWhenJobRemoved(jobID) => if(runningTasks.values.asScala.exists(_.getJobID == jobID)){ context.system.scheduler.scheduleOnce( diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerMessages.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerMessages.scala similarity index 89% rename from flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerMessages.scala rename to flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerMessages.scala index 974e4e8b57c72..32c3c5563a347 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerMessages.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerMessages.scala @@ -38,14 +38,7 @@ object TestingTaskManagerMessages { def asJava: java.util.Map[ExecutionAttemptID, Task] = tasks.asJava } - case class ResponseBroadcastVariablesWithReferences(number: Int) - - case object RequestNumActiveConnections - case class ResponseNumActiveConnections(number: Int) - case object RequestRunningTasks - - case object RequestBroadcastVariablesWithReferences case class NotifyWhenJobManagerTerminated(jobManager: ActorRef) @@ -81,14 +74,9 @@ object TestingTaskManagerMessages { // -------------------------------------------------------------------------- // Utility methods to allow simpler case object access from Java // -------------------------------------------------------------------------- - + def getRequestRunningTasksMessage: AnyRef = { RequestRunningTasks } - - def getRequestBroadcastVariablesWithReferencesMessage: AnyRef = { - RequestBroadcastVariablesWithReferences - } - } diff --git a/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala b/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala index ee1b2647be6e5..00410cc703199 100644 --- a/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala +++ b/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala @@ -22,9 +22,10 @@ import java.io._ import java.util.concurrent.TimeUnit import org.apache.flink.configuration.GlobalConfiguration -import org.apache.flink.test.util.{ForkableFlinkMiniCluster, TestBaseUtils} +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster +import org.apache.flink.test.util.TestBaseUtils import org.apache.flink.util.TestLogger -import org.junit.{AfterClass, BeforeClass, Test, Assert} +import org.junit.{AfterClass, Assert, BeforeClass, Test} import scala.concurrent.duration.FiniteDuration import scala.tools.nsc.Settings @@ -334,7 +335,7 @@ class ScalaShellITCase extends TestLogger { } object ScalaShellITCase { - var cluster: Option[ForkableFlinkMiniCluster] = None + var cluster: Option[LocalFlinkMiniCluster] = None val parallelism = 4 @BeforeClass diff --git a/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java b/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java index 8bb440cdbbf9f..f94ff68315702 100644 --- a/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java +++ b/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java @@ -40,6 +40,7 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; @@ -49,7 +50,6 @@ import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.TestStreamEnvironment; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.apache.flink.test.util.TestEnvironment; import org.junit.After; import org.junit.AfterClass; @@ -134,7 +134,7 @@ public void stop() { } } - private static ForkableFlinkMiniCluster flinkCluster; + private static LocalFlinkMiniCluster flinkCluster; // ------------------------------------------------------------------------ // Cluster Setup (Cassandra & Flink) @@ -205,7 +205,7 @@ public static void startFlink() throws Exception { Configuration config = new Configuration(); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 4); - flinkCluster = new ForkableFlinkMiniCluster(config); + flinkCluster = new LocalFlinkMiniCluster(config); flinkCluster.start(); } diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java index 9e3c33bbc51ce..c4949ff83096c 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java @@ -22,6 +22,7 @@ import org.apache.flink.api.java.typeutils.TypeInfoParser; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -30,7 +31,6 @@ import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper; import org.apache.flink.streaming.util.serialization.SimpleStringSchema; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.apache.flink.util.InstantiationUtil; import org.junit.AfterClass; @@ -58,7 +58,7 @@ public class KafkaShortRetentionTestBase implements Serializable { private static KafkaTestEnvironment kafkaServer; private static Properties standardProps; - private static ForkableFlinkMiniCluster flink; + private static LocalFlinkMiniCluster flink; @BeforeClass public static void prepare() throws IOException, ClassNotFoundException { @@ -88,7 +88,7 @@ public static void prepare() throws IOException, ClassNotFoundException { flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 16); flinkConfig.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "0 s"); - flink = new ForkableFlinkMiniCluster(flinkConfig, false); + flink = new LocalFlinkMiniCluster(flinkConfig, false); flink.start(); } diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java index eddb57c709e31..771db17120702 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java @@ -22,8 +22,8 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.jmx.JMXReporter; import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.apache.flink.test.util.SuccessException; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.TestLogger; @@ -65,7 +65,7 @@ public abstract class KafkaTestBase extends TestLogger { protected static Properties standardProps; - protected static ForkableFlinkMiniCluster flink; + protected static LocalFlinkMiniCluster flink; protected static int flinkPort; @@ -105,7 +105,7 @@ public static void prepare() throws IOException, ClassNotFoundException { flinkConfig.setString(ConfigConstants.METRICS_REPORTERS_LIST, "my_reporter"); flinkConfig.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "my_reporter." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, JMXReporter.class.getName()); - flink = new ForkableFlinkMiniCluster(flinkConfig, false); + flink = new LocalFlinkMiniCluster(flinkConfig, false); flink.start(); flinkPort = flink.getLeaderRPCPort(); diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceTest.java b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceTest.java index 37059437c39c2..2e452c1515205 100644 --- a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceTest.java +++ b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceTest.java @@ -21,11 +21,11 @@ import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants; import org.apache.flink.streaming.connectors.kinesis.testutils.ExactlyOnceValidatingConsumerThread; import org.apache.flink.streaming.connectors.kinesis.testutils.KinesisEventsGeneratorProducerThread; import org.apache.flink.streaming.connectors.kinesis.util.AWSUtil; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -80,7 +80,7 @@ public static void main(String[] args) throws Exception { flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 16); flinkConfig.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "0 s"); - ForkableFlinkMiniCluster flink = new ForkableFlinkMiniCluster(flinkConfig, false); + LocalFlinkMiniCluster flink = new LocalFlinkMiniCluster(flinkConfig, false); flink.start(); final int flinkPort = flink.getLeaderRPCPort(); diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceWithStreamReshardingTest.java b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceWithStreamReshardingTest.java index 934a7955735fe..6abea2a200e20 100644 --- a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceWithStreamReshardingTest.java +++ b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceWithStreamReshardingTest.java @@ -27,11 +27,11 @@ import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants; import org.apache.flink.streaming.connectors.kinesis.testutils.ExactlyOnceValidatingConsumerThread; import org.apache.flink.streaming.connectors.kinesis.testutils.KinesisShardIdGenerator; import org.apache.flink.streaming.connectors.kinesis.util.AWSUtil; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -92,7 +92,7 @@ public static void main(String[] args) throws Exception { flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 16); flinkConfig.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "0 s"); - ForkableFlinkMiniCluster flink = new ForkableFlinkMiniCluster(flinkConfig, false); + LocalFlinkMiniCluster flink = new LocalFlinkMiniCluster(flinkConfig, false); flink.start(); final int flinkPort = flink.getLeaderRPCPort(); diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/ScalaStreamingMultipleProgramsTestBase.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/ScalaStreamingMultipleProgramsTestBase.scala index ee415d165eefd..29b3a3e7cafbe 100644 --- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/ScalaStreamingMultipleProgramsTestBase.scala +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/ScalaStreamingMultipleProgramsTestBase.scala @@ -18,8 +18,9 @@ package org.apache.flink.streaming.api.scala +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster import org.apache.flink.streaming.util.TestStreamEnvironment -import org.apache.flink.test.util.{ForkableFlinkMiniCluster, TestBaseUtils} +import org.apache.flink.test.util.TestBaseUtils import org.scalatest.BeforeAndAfterAll import org.scalatest.junit.JUnitSuiteLike @@ -29,7 +30,7 @@ trait ScalaStreamingMultipleProgramsTestBase with BeforeAndAfterAll { val parallelism = 4 - var cluster: Option[ForkableFlinkMiniCluster] = None + var cluster: Option[LocalFlinkMiniCluster] = None override protected def beforeAll(): Unit = { val cluster = Some( diff --git a/flink-test-utils-parent/flink-test-utils/pom.xml b/flink-test-utils-parent/flink-test-utils/pom.xml index 2ab52b57e4065..18ecfded11d90 100644 --- a/flink-test-utils-parent/flink-test-utils/pom.xml +++ b/flink-test-utils-parent/flink-test-utils/pom.xml @@ -79,153 +79,4 @@ under the License. - - - - - - net.alchim31.maven - scala-maven-plugin - 3.1.4 - - - - scala-compile-first - process-resources - - compile - - - - - - scala-test-compile - process-test-resources - - testCompile - - - - - - -Xms128m - -Xmx512m - - - - - - - org.apache.maven.plugins - maven-eclipse-plugin - 2.8 - - true - - org.scala-ide.sdt.core.scalanature - org.eclipse.jdt.core.javanature - - - org.scala-ide.sdt.core.scalabuilder - - - org.scala-ide.sdt.launching.SCALA_CONTAINER - org.eclipse.jdt.launching.JRE_CONTAINER - - - org.scala-lang:scala-library - org.scala-lang:scala-compiler - - - **/*.scala - **/*.java - - - - - - - org.codehaus.mojo - build-helper-maven-plugin - 1.7 - - - - add-source - generate-sources - - add-source - - - - src/main/scala - - - - - - add-test-source - generate-test-sources - - add-test-source - - - - src/test/scala - - - - - - - - - org.scalastyle - scalastyle-maven-plugin - - ${project.basedir}/../../tools/maven/scalastyle-config.xml - - - - - - - - - org.eclipse.m2e - lifecycle-mapping - 1.0.0 - - - - - - - net.alchim31.maven - - - scala-maven-plugin - - - [3.1.4,) - - - compile - testCompile - - - - - - - - - - - - - diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/StreamingMultipleProgramsTestBase.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/StreamingMultipleProgramsTestBase.java index c5fbaf067aa3a..a47890877cfc9 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/StreamingMultipleProgramsTestBase.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/StreamingMultipleProgramsTestBase.java @@ -19,8 +19,8 @@ package org.apache.flink.streaming.util; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.test.util.AbstractTestBase; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.apache.flink.test.util.TestBaseUtils; import org.junit.AfterClass; @@ -61,7 +61,7 @@ public class StreamingMultipleProgramsTestBase extends AbstractTestBase { protected static final int DEFAULT_PARALLELISM = 4; - protected static ForkableFlinkMiniCluster cluster; + protected static LocalFlinkMiniCluster cluster; public StreamingMultipleProgramsTestBase() { super(new Configuration()); diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java index c70010259471b..64c68dc168a2e 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java @@ -20,10 +20,10 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironmentFactory; import org.apache.flink.streaming.api.graph.StreamGraph; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.apache.flink.util.Preconditions; /** @@ -32,10 +32,10 @@ public class TestStreamEnvironment extends StreamExecutionEnvironment { /** The mini cluster in which this environment executes its jobs */ - private ForkableFlinkMiniCluster executor; + private LocalFlinkMiniCluster executor; - public TestStreamEnvironment(ForkableFlinkMiniCluster executor, int parallelism) { + public TestStreamEnvironment(LocalFlinkMiniCluster executor, int parallelism) { this.executor = Preconditions.checkNotNull(executor); setParallelism(parallelism); } @@ -57,7 +57,7 @@ public JobExecutionResult execute(String jobName) throws Exception { * @param cluster The test cluster to run the test program on. * @param parallelism The default parallelism for the test programs. */ - public static void setAsContext(final ForkableFlinkMiniCluster cluster, final int parallelism) { + public static void setAsContext(final LocalFlinkMiniCluster cluster, final int parallelism) { StreamExecutionEnvironmentFactory factory = new StreamExecutionEnvironmentFactory() { @Override diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java index c2da69106eb9c..316fd21cbfbf7 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java @@ -21,6 +21,7 @@ import com.google.common.base.Charsets; import com.google.common.io.Files; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import scala.concurrent.duration.FiniteDuration; import java.io.File; @@ -48,7 +49,7 @@ public abstract class AbstractTestBase extends TestBaseUtils { protected int numTaskManagers = 1; /** The mini cluster that runs the test programs */ - protected ForkableFlinkMiniCluster executor; + protected LocalFlinkMiniCluster executor; public AbstractTestBase(Configuration config) { diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java index d7f09bdcaa8d3..4e83245a334b8 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java @@ -18,6 +18,7 @@ package org.apache.flink.test.util; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.runners.Parameterized; @@ -72,7 +73,7 @@ public enum TestExecutionMode { protected static boolean startWebServer = false; - protected static ForkableFlinkMiniCluster cluster = null; + protected static LocalFlinkMiniCluster cluster = null; // ------------------------------------------------------------------------ diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java index 4014b8052d5ba..b774f979ef8ef 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java @@ -32,7 +32,8 @@ import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages; +import org.apache.flink.runtime.messages.TaskManagerMessages; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.util.TestLogger; import org.apache.hadoop.fs.FileSystem; @@ -104,7 +105,7 @@ private static void verifyJvmOptions() { } - public static ForkableFlinkMiniCluster startCluster( + public static LocalFlinkMiniCluster startCluster( int numTaskManagers, int taskManagerNumSlots, boolean startWebserver, @@ -126,7 +127,7 @@ public static ForkableFlinkMiniCluster startCluster( return startCluster(config, singleActorSystem); } - public static ForkableFlinkMiniCluster startCluster( + public static LocalFlinkMiniCluster startCluster( Configuration config, boolean singleActorSystem) throws Exception { @@ -147,7 +148,7 @@ public static ForkableFlinkMiniCluster startCluster( config.setString(ConfigConstants.TASK_MANAGER_LOG_PATH_KEY, logFile.toString()); - ForkableFlinkMiniCluster cluster = new ForkableFlinkMiniCluster(config, singleActorSystem); + LocalFlinkMiniCluster cluster = new LocalFlinkMiniCluster(config, singleActorSystem); cluster.start(); @@ -155,7 +156,7 @@ public static ForkableFlinkMiniCluster startCluster( } - public static void stopCluster(ForkableFlinkMiniCluster executor, FiniteDuration timeout) throws Exception { + public static void stopCluster(LocalFlinkMiniCluster executor, FiniteDuration timeout) throws Exception { if (logDir != null) { FileUtils.deleteDirectory(logDir); } @@ -169,11 +170,15 @@ public static void stopCluster(ForkableFlinkMiniCluster executor, FiniteDuration List> numActiveConnectionsResponseFutures = new ArrayList<>(); for (ActorRef tm : tms) { - bcVariableManagerResponseFutures.add(Patterns.ask(tm, TestingTaskManagerMessages - .RequestBroadcastVariablesWithReferences$.MODULE$, new Timeout(timeout))); - - numActiveConnectionsResponseFutures.add(Patterns.ask(tm, TestingTaskManagerMessages - .RequestNumActiveConnections$.MODULE$, new Timeout(timeout))); + bcVariableManagerResponseFutures.add(Patterns.ask( + tm, + TaskManagerMessages.getRequestBroadcastVariablesWithReferences(), + new Timeout(timeout))); + + numActiveConnectionsResponseFutures.add(Patterns.ask( + tm, + TaskManagerMessages.getRequestNumActiveConnections(), + new Timeout(timeout))); } Future> bcVariableManagerFutureResponses = Futures.sequence( @@ -182,8 +187,7 @@ public static void stopCluster(ForkableFlinkMiniCluster executor, FiniteDuration Iterable responses = Await.result(bcVariableManagerFutureResponses, timeout); for (Object response : responses) { - numUnreleasedBCVars += ((TestingTaskManagerMessages - .ResponseBroadcastVariablesWithReferences) response).number(); + numUnreleasedBCVars += ((TaskManagerMessages.ResponseBroadcastVariablesWithReferences) response).number(); } Future> numActiveConnectionsFutureResponses = Futures.sequence( @@ -192,8 +196,7 @@ public static void stopCluster(ForkableFlinkMiniCluster executor, FiniteDuration responses = Await.result(numActiveConnectionsFutureResponses, timeout); for (Object response : responses) { - numActiveConnections += ((TestingTaskManagerMessages - .ResponseNumActiveConnections) response).number(); + numActiveConnections += ((TaskManagerMessages.ResponseNumActiveConnections) response).number(); } } diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java index 7cb88be34d714..aea81522150fb 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java @@ -29,10 +29,11 @@ import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator; import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; public class TestEnvironment extends ExecutionEnvironment { - private final ForkableFlinkMiniCluster executor; + private final LocalFlinkMiniCluster executor; private TestEnvironment lastEnv = null; @@ -46,7 +47,7 @@ public JobExecutionResult getLastJobExecutionResult() { } } - public TestEnvironment(ForkableFlinkMiniCluster executor, int parallelism) { + public TestEnvironment(LocalFlinkMiniCluster executor, int parallelism) { this.executor = executor; setParallelism(parallelism); @@ -54,7 +55,7 @@ public TestEnvironment(ForkableFlinkMiniCluster executor, int parallelism) { getConfig().setCodeAnalysisMode(CodeAnalysisMode.DISABLE); } - public TestEnvironment(ForkableFlinkMiniCluster executor, int parallelism, boolean isObjectReuseEnabled) { + public TestEnvironment(LocalFlinkMiniCluster executor, int parallelism, boolean isObjectReuseEnabled) { this(executor, parallelism); if (isObjectReuseEnabled) { diff --git a/flink-test-utils-parent/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala b/flink-test-utils-parent/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala deleted file mode 100644 index fa3135ad975e2..0000000000000 --- a/flink-test-utils-parent/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala +++ /dev/null @@ -1,335 +0,0 @@ -/* - * 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.test.util - -import java.util.concurrent.TimeoutException - -import akka.actor.{ActorRef, ActorSystem} -import akka.pattern.Patterns._ -import akka.pattern.ask - -import org.apache.curator.test.TestingCluster -import org.apache.flink.configuration.{ConfigConstants, Configuration} -import org.apache.flink.runtime.akka.AkkaUtils -import org.apache.flink.runtime.clusterframework.FlinkResourceManager -import org.apache.flink.runtime.clusterframework.types.ResourceID -import org.apache.flink.runtime.jobmanager.{JobManager, HighAvailabilityMode} -import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster -import org.apache.flink.runtime.taskmanager.TaskManager -import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages.NotifyWhenRegisteredAtJobManager -import org.apache.flink.runtime.testingUtils.{TestingJobManager, TestingMemoryArchivist, TestingTaskManager} -import org.apache.flink.runtime.testutils.TestingResourceManager - -import scala.concurrent.{Await, Future} -import scala.concurrent.duration._ - -/** - * A forkable mini cluster is a special case of the mini cluster, used for parallel test execution - * on build servers. If multiple tests run in parallel, the cluster picks up the fork number and - * uses it to avoid port conflicts. - * - * @param userConfiguration Configuration object with the user provided configuration values - * @param singleActorSystem true, if all actors (JobManager and TaskManager) shall be run in the - * same [[ActorSystem]], otherwise false. - */ -class ForkableFlinkMiniCluster( - userConfiguration: Configuration, - singleActorSystem: Boolean) - extends LocalFlinkMiniCluster(userConfiguration, singleActorSystem) { - - def this(userConfiguration: Configuration) = this(userConfiguration, true) - - // -------------------------------------------------------------------------- - - var zookeeperCluster: Option[TestingCluster] = None - - override def generateConfiguration(userConfiguration: Configuration): Configuration = { - val forkNumberString = System.getProperty("forkNumber") - - val forkNumber = try { - Integer.parseInt(forkNumberString) - } - catch { - case e: NumberFormatException => -1 - } - - val config = userConfiguration.clone() - - if (forkNumber != -1) { - val jobManagerRPC = 1024 + forkNumber*400 - val taskManagerRPC = 1024 + forkNumber*400 + 100 - val taskManagerData = 1024 + forkNumber*400 + 200 - val resourceManagerRPC = 1024 + forkNumber*400 + 300 - - config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerRPC) - config.setInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, taskManagerRPC) - config.setInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, taskManagerData) - config.setInteger(ConfigConstants.RESOURCE_MANAGER_IPC_PORT_KEY, resourceManagerRPC) - } - - super.generateConfiguration(config) - } - - override def startJobManager(index: Int, actorSystem: ActorSystem): ActorRef = { - val config = configuration.clone() - - val jobManagerName = getJobManagerName(index) - val archiveName = getArchiveName(index) - - val jobManagerPort = config.getInteger( - ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, - ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT) - - if (jobManagerPort > 0) { - config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerPort + index) - } - - val (jobManager, _) = JobManager.startJobManagerActors( - config, - actorSystem, - Some(jobManagerName), - Some(archiveName), - classOf[TestingJobManager], - classOf[TestingMemoryArchivist]) - - jobManager - } - - override def startResourceManager(index: Int, system: ActorSystem): ActorRef = { - val config = configuration.clone() - - val resourceManagerName = getResourceManagerName(index) - - val resourceManagerPort = config.getInteger( - ConfigConstants.RESOURCE_MANAGER_IPC_PORT_KEY, - ConfigConstants.DEFAULT_RESOURCE_MANAGER_IPC_PORT) - - if (resourceManagerPort > 0) { - config.setInteger(ConfigConstants.RESOURCE_MANAGER_IPC_PORT_KEY, resourceManagerPort + index) - } - - val resourceManager = FlinkResourceManager.startResourceManagerActors( - config, - system, - createLeaderRetrievalService(), - classOf[TestingResourceManager], - resourceManagerName) - - resourceManager - } - - override def startTaskManager(index: Int, system: ActorSystem): ActorRef = { - val config = configuration.clone() - - val rpcPort = config.getInteger( - ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, - ConfigConstants.DEFAULT_TASK_MANAGER_IPC_PORT) - - val dataPort = config.getInteger( - ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, - ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT) - - if (rpcPort > 0) { - config.setInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, rpcPort + index) - } - if (dataPort > 0) { - config.setInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, dataPort + index) - } - - val localExecution = numTaskManagers == 1 - - TaskManager.startTaskManagerComponentsAndActor( - config, - ResourceID.generate(), - system, - hostname, - Some(TaskManager.TASK_MANAGER_NAME + index), - Some(createLeaderRetrievalService()), - localExecution, - classOf[TestingTaskManager]) - } - - def addTaskManager(): Unit = { - if (useSingleActorSystem) { - (jobManagerActorSystems, taskManagerActors) match { - case (Some(jmSystems), Some(tmActors)) => - val index = numTaskManagers - taskManagerActors = Some(tmActors :+ startTaskManager(index, jmSystems(0))) - numTaskManagers += 1 - case _ => throw new IllegalStateException("Cluster has not been started properly.") - } - } else { - (taskManagerActorSystems, taskManagerActors) match { - case (Some(tmSystems), Some(tmActors)) => - val index = numTaskManagers - val newTmSystem = startTaskManagerActorSystem(index) - val newTmActor = startTaskManager(index, newTmSystem) - - taskManagerActorSystems = Some(tmSystems :+ newTmSystem) - taskManagerActors = Some(tmActors :+ newTmActor) - - numTaskManagers += 1 - case _ => throw new IllegalStateException("Cluster has not been started properly.") - } - } - } - - def restartLeadingJobManager(): Unit = { - this.synchronized { - (jobManagerActorSystems, jobManagerActors) match { - case (Some(jmActorSystems), Some(jmActors)) => - val leader = getLeaderGateway(AkkaUtils.getTimeout(configuration)) - val index = getLeaderIndex(AkkaUtils.getTimeout(configuration)) - - clearLeader() - - val stopped = gracefulStop(leader.actor(), ForkableFlinkMiniCluster.MAX_RESTART_DURATION) - Await.result(stopped, ForkableFlinkMiniCluster.MAX_RESTART_DURATION) - - if(!singleActorSystem) { - jmActorSystems(index).shutdown() - jmActorSystems(index).awaitTermination() - } - - val newJobManagerActorSystem = if(!singleActorSystem) { - startJobManagerActorSystem(index) - } else { - jmActorSystems.head - } - - val newJobManagerActor = startJobManager(index, newJobManagerActorSystem) - - jobManagerActors = Some(jmActors.patch(index, Seq(newJobManagerActor), 1)) - jobManagerActorSystems = Some(jmActorSystems.patch( - index, - Seq(newJobManagerActorSystem), - 1)) - - val lrs = createLeaderRetrievalService() - - jobManagerLeaderRetrievalService = Some(lrs) - lrs.start(this) - - case _ => throw new Exception("The JobManager of the ForkableFlinkMiniCluster have not " + - "been started properly.") - } - } - } - - - def restartTaskManager(index: Int): Unit = { - (taskManagerActorSystems, taskManagerActors) match { - case (Some(tmActorSystems), Some(tmActors)) => - val stopped = gracefulStop(tmActors(index), ForkableFlinkMiniCluster.MAX_RESTART_DURATION) - Await.result(stopped, ForkableFlinkMiniCluster.MAX_RESTART_DURATION) - - if(!singleActorSystem) { - tmActorSystems(index).shutdown() - tmActorSystems(index).awaitTermination() - } - - val taskManagerActorSystem = if(!singleActorSystem) { - startTaskManagerActorSystem(index) - } else { - tmActorSystems.head - } - - val taskManagerActor = startTaskManager(index, taskManagerActorSystem) - - taskManagerActors = Some(tmActors.patch(index, Seq(taskManagerActor), 1)) - taskManagerActorSystems = Some(tmActorSystems.patch(index, Seq(taskManagerActorSystem), 1)) - - case _ => throw new Exception("The TaskManager of the ForkableFlinkMiniCluster have not " + - "been started properly.") - } - } - - override def start(): Unit = { - val zookeeperURL = configuration.getString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, "") - - zookeeperCluster = if (haMode == HighAvailabilityMode.ZOOKEEPER && - zookeeperURL.equals("")) { - LOG.info("Starting ZooKeeper cluster.") - - val testingCluster = new TestingCluster(1) - - configuration.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, - testingCluster.getConnectString) - - testingCluster.start() - - Some(testingCluster) - } else { - None - } - - super.start() - } - - override def stop(): Unit = { - super.stop() - - zookeeperCluster.foreach{ - LOG.info("Stopping ZooKeeper cluster.") - _.close() - } - } - - def waitForTaskManagersToBeRegisteredAtJobManager(jobManager: ActorRef): Unit = { - val futures = taskManagerActors.map { - _.map { - tm => (tm ? NotifyWhenRegisteredAtJobManager(jobManager))(timeout) - } - }.getOrElse(Seq()) - - try { - Await.ready(Future.sequence(futures), timeout) - } catch { - case t: TimeoutException => - throw new Exception("Timeout while waiting for TaskManagers to register at " + - s"${jobManager.path}") - } - - } -} - -object ForkableFlinkMiniCluster { - - val MAX_RESTART_DURATION = 2 minute - - val DEFAULT_MINICLUSTER_AKKA_ASK_TIMEOUT = "200 s" - - def startCluster( - numSlots: Int, - numTaskManagers: Int, - timeout: String = DEFAULT_MINICLUSTER_AKKA_ASK_TIMEOUT) - : ForkableFlinkMiniCluster = { - - val config = new Configuration() - config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlots) - config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTaskManagers) - config.setString(ConfigConstants.AKKA_ASK_TIMEOUT, timeout) - - val cluster = new ForkableFlinkMiniCluster(config) - - cluster.start() - - cluster - } -} diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java index cac84512b4a4c..cc70fee7e29c7 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java @@ -28,7 +28,7 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.client.JobExecutionException; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -44,7 +44,7 @@ */ public class AccumulatorErrorITCase { - private static ForkableFlinkMiniCluster cluster; + private static LocalFlinkMiniCluster cluster; @BeforeClass public static void startCluster() { @@ -53,7 +53,7 @@ public static void startCluster() { config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 3); config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12); - cluster = new ForkableFlinkMiniCluster(config, false); + cluster = new LocalFlinkMiniCluster(config, false); cluster.start(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java index 49e18e04b69ec..624bfff2413cc 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java @@ -234,7 +234,6 @@ private static void verifyResults() { fail("Wrong accumulator results when map task begins execution."); } - int expectedAccVal = 0; /* for mapper task */ diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java index 9671fcee98fbe..8a08f15e051a3 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java @@ -22,8 +22,8 @@ import java.util.concurrent.TimeUnit; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.runtime.testingUtils.TestingUtils; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.apache.flink.util.TestLogger; import org.junit.Assert; @@ -39,7 +39,6 @@ import static org.apache.flink.runtime.taskmanager.TaskCancelTest.awaitRunning; import static org.apache.flink.runtime.taskmanager.TaskCancelTest.cancelJob; import org.apache.flink.runtime.testutils.JobManagerActorTestUtils; -import org.apache.flink.util.StringUtils; import org.apache.hadoop.fs.FileSystem; import org.junit.After; @@ -65,7 +64,7 @@ public abstract class CancelingTestBase extends TestLogger { // -------------------------------------------------------------------------------------------- - protected ForkableFlinkMiniCluster executor; + protected LocalFlinkMiniCluster executor; protected int taskManagerNumSlots = DEFAULT_TASK_MANAGER_NUM_SLOTS; @@ -88,7 +87,7 @@ public void startCluster() throws Exception { config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY, 4096); config.setInteger(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, 2048); - this.executor = new ForkableFlinkMiniCluster(config, false); + this.executor = new LocalFlinkMiniCluster(config, false); this.executor.start(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java index 163fb424a7c30..94ff66f277055 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java @@ -25,6 +25,7 @@ import org.apache.flink.api.java.tuple.Tuple4; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.runtime.state.CheckpointListener; import org.apache.flink.streaming.api.checkpoint.Checkpointed; @@ -35,7 +36,6 @@ import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.streaming.api.windowing.windows.TimeWindow; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.apache.flink.test.util.SuccessException; import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; @@ -60,7 +60,7 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger { private static final int PARALLELISM = 4; - private static ForkableFlinkMiniCluster cluster; + private static LocalFlinkMiniCluster cluster; @BeforeClass @@ -71,7 +71,7 @@ public static void startTestCluster() { config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 48); config.setString(ConfigConstants.DEFAULT_AKKA_LOOKUP_TIMEOUT, "60 s"); config.setString(ConfigConstants.DEFAULT_AKKA_ASK_TIMEOUT, "60 s"); - cluster = new ForkableFlinkMiniCluster(config, false); + cluster = new LocalFlinkMiniCluster(config, false); cluster.start(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java index fa5339d90eef8..0aee128a24042 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java @@ -28,6 +28,7 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.contrib.streaming.state.RocksDBStateBackend; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.runtime.state.AbstractStateBackend; import org.apache.flink.runtime.state.filesystem.FsStateBackend; import org.apache.flink.runtime.state.memory.MemoryStateBackend; @@ -41,7 +42,6 @@ import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.streaming.api.windowing.windows.TimeWindow; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.apache.flink.test.util.SuccessException; import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; @@ -76,7 +76,7 @@ public class EventTimeWindowCheckpointingITCase extends TestLogger { private static final int MAX_MEM_STATE_SIZE = 10 * 1024 * 1024; private static final int PARALLELISM = 4; - private static ForkableFlinkMiniCluster cluster; + private static LocalFlinkMiniCluster cluster; @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); @@ -95,7 +95,7 @@ public static void startTestCluster() { config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM / 2); config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 48); - cluster = new ForkableFlinkMiniCluster(config, false); + cluster = new LocalFlinkMiniCluster(config, false); cluster.start(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java index 8915bfffd3354..7f1d7f3360f78 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java @@ -36,6 +36,7 @@ import org.apache.flink.runtime.messages.JobManagerMessages; import org.apache.flink.runtime.state.KeyGroupRangeAssignment; import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory; +import org.apache.flink.runtime.testingUtils.TestingCluster; import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages; import org.apache.flink.streaming.api.checkpoint.Checkpointed; import org.apache.flink.streaming.api.datastream.DataStream; @@ -43,7 +44,6 @@ import org.apache.flink.streaming.api.functions.sink.DiscardingSink; import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; import org.junit.AfterClass; @@ -73,7 +73,7 @@ public class RescalingITCase extends TestLogger { private static int slotsPerTaskManager = 2; private static int numSlots = numTaskManagers * slotsPerTaskManager; - private static ForkableFlinkMiniCluster cluster; + private static TestingCluster cluster; @ClassRule public static TemporaryFolder temporaryFolder = new TemporaryFolder(); @@ -92,7 +92,7 @@ public static void setup() throws Exception { config.setString(SavepointStoreFactory.SAVEPOINT_BACKEND_KEY, "filesystem"); config.setString(SavepointStoreFactory.SAVEPOINT_DIRECTORY_KEY, savepointDir.toURI().toString()); - cluster = new ForkableFlinkMiniCluster(config); + cluster = new TestingCluster(config); cluster.start(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java index 550ba75a367ac..7409fe7e3f875 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java @@ -43,7 +43,6 @@ import org.apache.flink.runtime.messages.JobManagerMessages.CancelJob; import org.apache.flink.runtime.messages.JobManagerMessages.DisposeSavepoint; import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepoint; -import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepointFailure; import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepointSuccess; import org.apache.flink.runtime.state.ChainedStateHandle; import org.apache.flink.runtime.state.CheckpointListener; @@ -51,6 +50,7 @@ import org.apache.flink.runtime.state.filesystem.FileStateHandle; import org.apache.flink.runtime.state.filesystem.FsStateBackend; import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory; +import org.apache.flink.runtime.testingUtils.TestingCluster; import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.NotifyWhenJobRemoved; import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.RequestSavepoint; import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.ResponseSavepoint; @@ -62,8 +62,6 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.DiscardingSink; import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; -import org.apache.flink.testutils.junit.RetryOnFailure; import org.apache.flink.testutils.junit.RetryRule; import org.apache.flink.util.TestLogger; import org.junit.Rule; @@ -76,7 +74,6 @@ import scala.concurrent.duration.FiniteDuration; import java.io.File; -import java.io.FileNotFoundException; import java.io.Serializable; import java.util.ArrayList; import java.util.Arrays; @@ -137,7 +134,7 @@ public void testTriggerSavepointAndResume() throws Exception { LOG.info("Created temporary directory: " + tmpDir + "."); - ForkableFlinkMiniCluster flink = null; + TestingCluster flink = null; try { // Create a test actor system @@ -168,7 +165,7 @@ public void testTriggerSavepointAndResume() throws Exception { LOG.info("Flink configuration: " + config + "."); // Start Flink - flink = new ForkableFlinkMiniCluster(config); + flink = new TestingCluster(config); LOG.info("Starting Flink cluster."); flink.start(); @@ -261,7 +258,7 @@ public void testTriggerSavepointAndResume() throws Exception { LOG.info("JobManager: " + jobManager + "."); final Throwable[] error = new Throwable[1]; - final ForkableFlinkMiniCluster finalFlink = flink; + final TestingCluster finalFlink = flink; final Multimap tdds = HashMultimap.create(); new JavaTestKit(testActorSystem) {{ @@ -422,7 +419,7 @@ public void testCheckpointsRemovedWithJobManagerBackendOnShutdown() throws Excep LOG.info("Created temporary directory: " + tmpDir + "."); - ForkableFlinkMiniCluster flink = null; + TestingCluster flink = null; List checkpointFiles = new ArrayList<>(); try { @@ -447,7 +444,7 @@ public void testCheckpointsRemovedWithJobManagerBackendOnShutdown() throws Excep LOG.info("Flink configuration: " + config + "."); // Start Flink - flink = new ForkableFlinkMiniCluster(config); + flink = new TestingCluster(config); LOG.info("Starting Flink cluster."); flink.start(); @@ -559,7 +556,7 @@ public void testSubmitWithUnknownSavepointPath() throws Exception { // Test deadline final Deadline deadline = new FiniteDuration(5, TimeUnit.MINUTES).fromNow(); - ForkableFlinkMiniCluster flink = null; + TestingCluster flink = null; try { // Flink configuration @@ -570,7 +567,7 @@ public void testSubmitWithUnknownSavepointPath() throws Exception { LOG.info("Flink configuration: " + config + "."); // Start Flink - flink = new ForkableFlinkMiniCluster(config); + flink = new TestingCluster(config); LOG.info("Starting Flink cluster."); flink.start(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java index cf15052b6297e..6bf511fbb2ce8 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java @@ -24,6 +24,7 @@ import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.runtime.state.CheckpointListener; import org.apache.flink.streaming.api.checkpoint.Checkpointed; import org.apache.flink.streaming.api.datastream.DataStream; @@ -34,7 +35,6 @@ import org.apache.flink.streaming.api.functions.source.RichSourceFunction; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; @@ -80,7 +80,7 @@ public class StreamCheckpointNotifierITCase extends TestLogger { private static final int NUM_TASK_SLOTS = 3; private static final int PARALLELISM = NUM_TASK_MANAGERS * NUM_TASK_SLOTS; - private static ForkableFlinkMiniCluster cluster; + private static LocalFlinkMiniCluster cluster; @BeforeClass public static void startCluster() { @@ -91,7 +91,7 @@ public static void startCluster() { config.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "0 ms"); config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12); - cluster = new ForkableFlinkMiniCluster(config, false); + cluster = new LocalFlinkMiniCluster(config, false); cluster.start(); } catch (Exception e) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java index 67c05e556e08a..5f6cd4aa86382 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java @@ -20,8 +20,8 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.apache.flink.test.util.TestUtils; import org.apache.flink.util.TestLogger; @@ -43,7 +43,7 @@ public abstract class StreamFaultToleranceTestBase extends TestLogger { protected static final int NUM_TASK_SLOTS = 4; protected static final int PARALLELISM = NUM_TASK_MANAGERS * NUM_TASK_SLOTS; - private static ForkableFlinkMiniCluster cluster; + private static LocalFlinkMiniCluster cluster; @BeforeClass public static void startCluster() { @@ -53,7 +53,7 @@ public static void startCluster() { config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_TASK_SLOTS); config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12); - cluster = new ForkableFlinkMiniCluster(config, false); + cluster = new LocalFlinkMiniCluster(config, false); cluster.start(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java index 2e6ce78948d19..e424a8d4d9605 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java @@ -25,6 +25,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.runtime.state.CheckpointListener; import org.apache.flink.streaming.api.checkpoint.Checkpointed; @@ -34,7 +35,6 @@ import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction; import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.streaming.api.windowing.windows.TimeWindow; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.apache.flink.test.util.SuccessException; import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; @@ -71,7 +71,7 @@ public WindowCheckpointingITCase(TimeCharacteristic timeCharacteristic) { private static final int PARALLELISM = 4; - private static ForkableFlinkMiniCluster cluster; + private static LocalFlinkMiniCluster cluster; @BeforeClass @@ -81,7 +81,7 @@ public static void startTestCluster() { config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM / 2); config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 48); - cluster = new ForkableFlinkMiniCluster(config, false); + cluster = new LocalFlinkMiniCluster(config, false); cluster.start(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java index 8b56d3d1b7ca2..7afafe43274ae 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java @@ -18,7 +18,6 @@ package org.apache.flink.test.classloading; -import akka.pattern.AskTimeoutException; import org.apache.flink.api.common.JobID; import org.apache.flink.client.program.PackagedProgram; import org.apache.flink.client.program.ProgramInvocationException; @@ -37,9 +36,9 @@ import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepoint; import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepointSuccess; import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory; +import org.apache.flink.runtime.testingUtils.TestingCluster; import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.WaitForAllVerticesToBeRunning; import org.apache.flink.test.testdata.KMeansData; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.apache.flink.util.TestLogger; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -82,7 +81,7 @@ public class ClassLoaderITCase extends TestLogger { public static final TemporaryFolder FOLDER = new TemporaryFolder(); - private static ForkableFlinkMiniCluster testCluster; + private static TestingCluster testCluster; private static int parallelism; @@ -105,7 +104,7 @@ public static void setUp() throws Exception { config.setString(SavepointStoreFactory.SAVEPOINT_DIRECTORY_KEY, FOLDER.newFolder().getAbsoluteFile().toURI().toString()); - testCluster = new ForkableFlinkMiniCluster(config, false); + testCluster = new TestingCluster(config, false); testCluster.start(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/clients/examples/JobRetrievalITCase.java b/flink-tests/src/test/java/org/apache/flink/test/clients/examples/JobRetrievalITCase.java index c9059f1f8936b..a74ed34e02286 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/clients/examples/JobRetrievalITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/clients/examples/JobRetrievalITCase.java @@ -29,8 +29,8 @@ import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.minicluster.FlinkMiniCluster; +import org.apache.flink.runtime.testingUtils.TestingCluster; import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.apache.flink.util.TestLogger; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -42,7 +42,6 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.fail; - /** * Tests retrieval of a job from a running Flink cluster */ @@ -54,7 +53,7 @@ public class JobRetrievalITCase extends TestLogger { @BeforeClass public static void before() { - cluster = new ForkableFlinkMiniCluster(new Configuration(), false); + cluster = new TestingCluster(new Configuration(), false); cluster.start(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java index 28c2e5837fc95..178656dc7e9ce 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java @@ -26,8 +26,8 @@ import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.Tasks; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.runtime.testingUtils.TestingUtils; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -47,7 +47,7 @@ public class JobSubmissionFailsITCase { private static final int NUM_SLOTS = 20; - private static ForkableFlinkMiniCluster cluster; + private static LocalFlinkMiniCluster cluster; private static JobGraph workingJobGraph; @BeforeClass @@ -58,7 +58,7 @@ public static void setup() { config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_SLOTS / 2); - cluster = new ForkableFlinkMiniCluster(config); + cluster = new LocalFlinkMiniCluster(config); cluster.start(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CustomDistributionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CustomDistributionITCase.java index ca2c15620d8fd..133ebd0920b6d 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CustomDistributionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CustomDistributionITCase.java @@ -29,8 +29,8 @@ import org.apache.flink.api.java.utils.DataSetUtils; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.apache.flink.test.util.TestBaseUtils; import org.apache.flink.test.util.TestEnvironment; import org.apache.flink.util.Collector; @@ -52,7 +52,7 @@ public class CustomDistributionITCase extends TestLogger { // The mini cluster that is shared across tests // ------------------------------------------------------------------------ - private static ForkableFlinkMiniCluster cluster; + private static LocalFlinkMiniCluster cluster; @BeforeClass public static void setup() throws Exception { diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/RemoteEnvironmentITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/RemoteEnvironmentITCase.java index 34a7eedaa9d3e..e18e82a151b87 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/RemoteEnvironmentITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/RemoteEnvironmentITCase.java @@ -23,11 +23,10 @@ import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.io.LocalCollectionOutputFormat; -import org.apache.flink.client.program.ProgramInvocationException; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.io.GenericInputSplit; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.util.Collector; import org.junit.AfterClass; import org.junit.Assert; @@ -54,7 +53,7 @@ public class RemoteEnvironmentITCase { private static final String VALID_STARTUP_TIMEOUT = "100 s"; - private static ForkableFlinkMiniCluster cluster; + private static LocalFlinkMiniCluster cluster; @BeforeClass public static void setupCluster() { @@ -62,7 +61,7 @@ public static void setupCluster() { Configuration config = new Configuration(); config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TM); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, TM_SLOTS); - cluster = new ForkableFlinkMiniCluster(config, false); + cluster = new LocalFlinkMiniCluster(config, false); cluster.start(); } catch (Exception e) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java index 09b5e7eb68677..a67e6ef6b56bb 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java @@ -29,7 +29,7 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.io.GenericInputSplit; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.util.Collector; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -49,14 +49,14 @@ public class AutoParallelismITCase { private static final int SLOTS_PER_TM = 7; private static final int PARALLELISM = NUM_TM * SLOTS_PER_TM; - private static ForkableFlinkMiniCluster cluster; + private static LocalFlinkMiniCluster cluster; @BeforeClass public static void setupCluster() { Configuration config = new Configuration(); config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TM); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, SLOTS_PER_TM); - cluster = new ForkableFlinkMiniCluster(config, false); + cluster = new LocalFlinkMiniCluster(config, false); cluster.start(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/CustomSerializationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/CustomSerializationITCase.java index f30f61fc3a57d..51f35344b4add 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/misc/CustomSerializationITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/misc/CustomSerializationITCase.java @@ -26,7 +26,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.types.Value; import org.junit.AfterClass; @@ -43,7 +43,7 @@ public class CustomSerializationITCase { private static final int PARLLELISM = 5; - private static ForkableFlinkMiniCluster cluster; + private static LocalFlinkMiniCluster cluster; @BeforeClass public static void startCluster() { @@ -51,7 +51,7 @@ public static void startCluster() { Configuration config = new Configuration(); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARLLELISM); config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 30); - cluster = new ForkableFlinkMiniCluster(config, false); + cluster = new LocalFlinkMiniCluster(config, false); cluster.start(); } catch (Exception e) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java index 42419fb94307f..06b93ea9dd58a 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java @@ -29,7 +29,7 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.FileSystem; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.util.Collector; @@ -52,7 +52,7 @@ @SuppressWarnings("serial") public class MiscellaneousIssuesITCase { - private static ForkableFlinkMiniCluster cluster; + private static LocalFlinkMiniCluster cluster; @BeforeClass public static void startCluster() { @@ -61,7 +61,7 @@ public static void startCluster() { config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 3); config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12); - cluster = new ForkableFlinkMiniCluster(config, false); + cluster = new LocalFlinkMiniCluster(config, false); cluster.start(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java index 12b7a68f67858..a43bab64fbd8d 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java @@ -32,8 +32,8 @@ import org.apache.flink.examples.java.clustering.util.KMeansData; import org.apache.flink.examples.java.graph.ConnectedComponents; import org.apache.flink.examples.java.graph.util.ConnectedComponentsData; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.junit.Test; import static org.junit.Assert.*; @@ -43,7 +43,7 @@ public class SuccessAfterNetworkBuffersFailureITCase { @Test public void testSuccessfulProgramAfterFailure() { - ForkableFlinkMiniCluster cluster = null; + LocalFlinkMiniCluster cluster = null; try { Configuration config = new Configuration(); @@ -52,7 +52,7 @@ public void testSuccessfulProgramAfterFailure() { config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 8); config.setInteger(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, 840); - cluster = new ForkableFlinkMiniCluster(config, false); + cluster = new LocalFlinkMiniCluster(config, false); cluster.start(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/query/QueryableStateITCase.java b/flink-tests/src/test/java/org/apache/flink/test/query/QueryableStateITCase.java index 40732dfec2859..b99858a0dd40f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/query/QueryableStateITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/query/QueryableStateITCase.java @@ -55,6 +55,7 @@ import org.apache.flink.runtime.state.CheckpointListener; import org.apache.flink.runtime.state.VoidNamespace; import org.apache.flink.runtime.state.VoidNamespaceSerializer; +import org.apache.flink.runtime.testingUtils.TestingCluster; import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages; import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages; import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages.ResponseRunningTasks; @@ -62,7 +63,6 @@ import org.apache.flink.streaming.api.datastream.QueryableStateStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.apache.flink.util.MathUtils; import org.apache.flink.util.Preconditions; import org.apache.flink.util.TestLogger; @@ -108,7 +108,7 @@ public class QueryableStateITCase extends TestLogger { * Shared between all the test. Make sure to have at least NUM_SLOTS * available after your test finishes, e.g. cancel the job you submitted. */ - private static ForkableFlinkMiniCluster cluster; + private static TestingCluster cluster; @BeforeClass public static void setup() { @@ -120,7 +120,7 @@ public static void setup() { config.setInteger(ConfigConstants.QUERYABLE_STATE_CLIENT_NETWORK_THREADS, 1); config.setInteger(ConfigConstants.QUERYABLE_STATE_SERVER_NETWORK_THREADS, 1); - cluster = new ForkableFlinkMiniCluster(config, false); + cluster = new TestingCluster(config, false); cluster.start(true); } catch (Exception e) { e.printStackTrace(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java index 8a45d6259d37b..8a43ee4026315 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java @@ -24,11 +24,11 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.functions.source.RichSourceFunction; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -49,7 +49,7 @@ public void testThis() { config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 2); - ForkableFlinkMiniCluster cluster = new ForkableFlinkMiniCluster(config, false); + LocalFlinkMiniCluster cluster = new LocalFlinkMiniCluster(config, false); cluster.start(); StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment( diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFailureRateStrategyITBase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFailureRateStrategyITBase.java index 0c5d14b23a9e6..a0d6b58f127c1 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFailureRateStrategyITBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFailureRateStrategyITBase.java @@ -20,7 +20,7 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.junit.BeforeClass; public class SimpleRecoveryFailureRateStrategyITBase extends SimpleRecoveryITCaseBase { @@ -34,8 +34,8 @@ public static void setupCluster() { config.setString(ConfigConstants.RESTART_STRATEGY_FAILURE_RATE_FAILURE_RATE_INTERVAL, "1 second"); config.setString(ConfigConstants.RESTART_STRATEGY_FAILURE_RATE_DELAY, "100 ms"); - cluster = new ForkableFlinkMiniCluster(config, false); + cluster = new LocalFlinkMiniCluster(config, false); cluster.start(); } -} \ No newline at end of file +} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFixedDelayRestartStrategyITBase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFixedDelayRestartStrategyITBase.java index 6355a8f39613a..f09efc510b9a1 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFixedDelayRestartStrategyITBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFixedDelayRestartStrategyITBase.java @@ -20,7 +20,7 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.junit.BeforeClass; public class SimpleRecoveryFixedDelayRestartStrategyITBase extends SimpleRecoveryITCaseBase { @@ -33,8 +33,8 @@ public static void setupCluster() { config.setInteger(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_ATTEMPTS, 1); config.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "100 ms"); - cluster = new ForkableFlinkMiniCluster(config, false); + cluster = new LocalFlinkMiniCluster(config, false); cluster.start(); } -} \ No newline at end of file +} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCaseBase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCaseBase.java index 004340ca2ee0d..bf7c524796203 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCaseBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCaseBase.java @@ -26,7 +26,7 @@ import org.apache.flink.api.java.io.LocalCollectionOutputFormat; import org.apache.flink.client.program.ProgramInvocationException; import org.apache.flink.runtime.client.JobExecutionException; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.junit.AfterClass; import org.junit.Test; @@ -42,7 +42,7 @@ @SuppressWarnings("serial") public abstract class SimpleRecoveryITCaseBase { - protected static ForkableFlinkMiniCluster cluster; + protected static LocalFlinkMiniCluster cluster; @AfterClass public static void teardownCluster() { diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java index 6c621accd8d68..5d2990528cc4f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java @@ -30,7 +30,7 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.messages.TaskManagerMessages; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.junit.Test; import scala.concurrent.Await; import scala.concurrent.Future; @@ -65,7 +65,7 @@ public void testRestartWithFailingTaskManager() { final int PARALLELISM = 4; - ForkableFlinkMiniCluster cluster = null; + LocalFlinkMiniCluster cluster = null; ActorSystem additionalSystem = null; try { @@ -78,7 +78,7 @@ public void testRestartWithFailingTaskManager() { config.setString(ConfigConstants.AKKA_WATCH_HEARTBEAT_PAUSE, "20 s"); config.setInteger(ConfigConstants.AKKA_WATCH_THRESHOLD, 20); - cluster = new ForkableFlinkMiniCluster(config, false); + cluster = new LocalFlinkMiniCluster(config, false); cluster.start(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java index 7710f06915357..0b008eb7378a1 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java @@ -26,8 +26,8 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.test.testdata.WordCountData; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.apache.flink.test.util.TestBaseUtils; import org.apache.flink.util.Collector; @@ -63,7 +63,7 @@ public void testClusterWithIPv6host() { - ForkableFlinkMiniCluster flink = null; + LocalFlinkMiniCluster flink = null; try { final String addressString = ipv6address.getHostAddress(); log.info("Test will use IPv6 address " + addressString + " for connection tests"); @@ -75,7 +75,7 @@ public void testClusterWithIPv6host() { conf.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 2); conf.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 16); - flink = new ForkableFlinkMiniCluster(conf, false); + flink = new LocalFlinkMiniCluster(conf, false); flink.start(); ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(addressString, flink.getLeaderRPCPort()); diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java index 9bd8cc3c994a1..2738d2259061a 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java @@ -21,7 +21,7 @@ import akka.actor.ActorSystem; import akka.actor.Kill; import akka.actor.PoisonPill; -import org.apache.commons.io.FileUtils; +import org.apache.curator.test.TestingServer; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -35,14 +35,19 @@ import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; +import org.apache.flink.runtime.testingUtils.TestingCluster; import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages; import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.WaitForAllVerticesToBeRunningOrFinished; import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.runtime.testutils.ZooKeeperTestUtils; import org.apache.flink.runtime.util.LeaderRetrievalUtils; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.apache.flink.util.TestLogger; import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import scala.concurrent.Await; @@ -52,7 +57,6 @@ import scala.concurrent.impl.Promise; import java.io.File; -import java.io.IOException; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; @@ -61,22 +65,20 @@ public class ZooKeeperLeaderElectionITCase extends TestLogger { private static final FiniteDuration timeout = TestingUtils.TESTING_DURATION(); - private static final File tempDirectory; + private static TestingServer zkServer; - static { - try { - tempDirectory = org.apache.flink.runtime.testutils - .CommonTestUtils.createTempDirectory(); - } - catch (IOException e) { - throw new RuntimeException("Test setup failed", e); - } + @Rule + public TemporaryFolder tempFolder = new TemporaryFolder(); + + @BeforeClass + public static void setup() throws Exception { + zkServer = new TestingServer(true); } @AfterClass public static void tearDown() throws Exception { - if (tempDirectory != null) { - FileUtils.deleteDirectory(tempDirectory); + if (zkServer != null) { + zkServer.close(); } } @@ -86,18 +88,19 @@ public static void tearDown() throws Exception { */ @Test public void testTaskManagerRegistrationAtReelectedLeader() throws Exception { - Configuration configuration = new Configuration(); + File rootFolder = tempFolder.getRoot(); + + Configuration configuration = ZooKeeperTestUtils.createZooKeeperHAConfig( + zkServer.getConnectString(), + rootFolder.getPath()); int numJMs = 10; int numTMs = 3; - configuration.setString(ConfigConstants.HA_MODE, "zookeeper"); configuration.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, numJMs); configuration.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTMs); - configuration.setString(ConfigConstants.STATE_BACKEND, "filesystem"); - configuration.setString(ConfigConstants.HA_ZOOKEEPER_STORAGE_PATH, tempDirectory.getAbsoluteFile().toURI().toString()); - ForkableFlinkMiniCluster cluster = new ForkableFlinkMiniCluster(configuration); + TestingCluster cluster = new TestingCluster(configuration); try { cluster.start(); @@ -137,14 +140,15 @@ public void testJobExecutionOnClusterWithLeaderReelection() throws Exception { int numSlotsPerTM = 3; int parallelism = numTMs * numSlotsPerTM; - Configuration configuration = new Configuration(); + File rootFolder = tempFolder.getRoot(); + + Configuration configuration = ZooKeeperTestUtils.createZooKeeperHAConfig( + zkServer.getConnectString(), + rootFolder.getPath()); - configuration.setString(ConfigConstants.HA_MODE, "zookeeper"); configuration.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, numJMs); configuration.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTMs); configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlotsPerTM); - configuration.setString(ConfigConstants.STATE_BACKEND, "filesystem"); - configuration.setString(ConfigConstants.HA_ZOOKEEPER_STORAGE_PATH, tempDirectory.getAbsoluteFile().toURI().toString()); // we "effectively" disable the automatic RecoverAllJobs message and sent it manually to make // sure that all TMs have registered to the JM prior to issueing the RecoverAllJobs message @@ -169,7 +173,7 @@ public void testJobExecutionOnClusterWithLeaderReelection() throws Exception { final JobGraph graph = new JobGraph("Blocking test job", sender, receiver); - final ForkableFlinkMiniCluster cluster = new ForkableFlinkMiniCluster(configuration); + final TestingCluster cluster = new TestingCluster(configuration); ActorSystem clientActorSystem = null; @@ -250,14 +254,14 @@ public static class JobSubmitterRunnable implements Runnable { boolean finished = false; final ActorSystem clientActorSystem; - final ForkableFlinkMiniCluster cluster; + final LocalFlinkMiniCluster cluster; final JobGraph graph; final Promise resultPromise = new Promise.DefaultPromise<>(); public JobSubmitterRunnable( ActorSystem actorSystem, - ForkableFlinkMiniCluster cluster, + LocalFlinkMiniCluster cluster, JobGraph graph) { this.clientActorSystem = actorSystem; this.cluster = cluster; diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/TimestampITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/TimestampITCase.java index d693aaaedad3f..2ed759d0b4e0f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/TimestampITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/TimestampITCase.java @@ -27,6 +27,7 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.testutils.MultiShotLatch; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -43,7 +44,6 @@ import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.apache.flink.util.TestLogger; import org.apache.flink.util.TestLogger; @@ -75,7 +75,7 @@ public class TimestampITCase extends TestLogger { static MultiShotLatch latch; - private static ForkableFlinkMiniCluster cluster; + private static LocalFlinkMiniCluster cluster; @Before public void setupLatch() { @@ -92,7 +92,7 @@ public static void startCluster() { config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_TASK_SLOTS); config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12); - cluster = new ForkableFlinkMiniCluster(config, false); + cluster = new LocalFlinkMiniCluster(config, false); cluster.start(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java b/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java index fc90994c3b070..a8482acd85927 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java @@ -29,11 +29,11 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.runtime.testutils.StoppableInvokable; import org.apache.flink.runtime.webmonitor.WebMonitorUtils; import org.apache.flink.runtime.webmonitor.files.MimeTypes; import org.apache.flink.runtime.webmonitor.testutils.HttpTestClient; -import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.apache.flink.test.util.TestBaseUtils; import org.apache.flink.util.TestLogger; @@ -62,7 +62,7 @@ public class WebFrontendITCase extends TestLogger { private static final int NUM_TASK_MANAGERS = 2; private static final int NUM_SLOTS = 4; - private static ForkableFlinkMiniCluster cluster; + private static LocalFlinkMiniCluster cluster; private static int port = -1; @@ -86,7 +86,7 @@ public static void initialize() throws Exception { config.setString(ConfigConstants.JOB_MANAGER_WEB_LOG_PATH_KEY, logFile.getAbsolutePath()); config.setString(ConfigConstants.TASK_MANAGER_LOG_PATH_KEY, logFile.getAbsolutePath()); - cluster = new ForkableFlinkMiniCluster(config, false); + cluster = new LocalFlinkMiniCluster(config, false); cluster.start(); port = cluster.webMonitor().get().getServerPort(); diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerFailsITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerFailsITCase.scala index ac661f3e727d2..1b2838db3a4ed 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerFailsITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerFailsITCase.scala @@ -21,7 +21,6 @@ package org.apache.flink.api.scala.runtime.jobmanager import akka.actor.{ActorSystem, PoisonPill} import akka.testkit.{ImplicitSender, TestKit} import org.apache.flink.configuration.{ConfigConstants, Configuration} -import org.apache.flink.api.common.{ExecutionConfig, ExecutionConfigTest} import org.apache.flink.runtime.akka.{AkkaUtils, ListeningBehaviour} import org.apache.flink.runtime.jobgraph.{JobGraph, JobVertex} import org.apache.flink.runtime.jobmanager.Tasks.{BlockingNoOpInvokable, NoOpInvokable} @@ -30,8 +29,7 @@ import org.apache.flink.runtime.messages.Messages.Acknowledge import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.NotifyWhenAtLeastNumTaskManagerAreRegistered import org.apache.flink.runtime.testingUtils.TestingMessages.DisableDisconnect import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages.{JobManagerTerminated, NotifyWhenJobManagerTerminated} -import org.apache.flink.runtime.testingUtils.{ScalaTestingUtils, TestingUtils} -import org.apache.flink.test.util.ForkableFlinkMiniCluster +import org.apache.flink.runtime.testingUtils.{ScalaTestingUtils, TestingCluster, TestingUtils} import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner import org.scalatest.{BeforeAndAfterAll, Matchers, WordSpecLike} @@ -140,12 +138,12 @@ class JobManagerFailsITCase(_system: ActorSystem) } } - def startDeathwatchCluster(numSlots: Int, numTaskmanagers: Int): ForkableFlinkMiniCluster = { + def startDeathwatchCluster(numSlots: Int, numTaskmanagers: Int): TestingCluster = { val config = new Configuration() config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlots) config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTaskmanagers) - val cluster = new ForkableFlinkMiniCluster(config, singleActorSystem = false) + val cluster = new TestingCluster(config, singleActorSystem = false) cluster.start() diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/taskmanager/TaskManagerFailsITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/taskmanager/TaskManagerFailsITCase.scala index 258f6dfceae04..3b39b3fe33a59 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/taskmanager/TaskManagerFailsITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/taskmanager/TaskManagerFailsITCase.scala @@ -20,7 +20,6 @@ package org.apache.flink.api.scala.runtime.taskmanager import akka.actor.{ActorSystem, Kill, PoisonPill} import akka.testkit.{ImplicitSender, TestKit} -import org.apache.flink.api.common.{ExecutionConfig, ExecutionConfigTest} import org.apache.flink.configuration.ConfigConstants import org.apache.flink.configuration.Configuration import org.apache.flink.runtime.akka.{AkkaUtils, ListeningBehaviour} @@ -31,8 +30,7 @@ import org.apache.flink.runtime.messages.JobManagerMessages._ import org.apache.flink.runtime.messages.TaskManagerMessages.{NotifyWhenRegisteredAtJobManager, RegisteredAtJobManager} import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages._ import org.apache.flink.runtime.testingUtils.TestingMessages.DisableDisconnect -import org.apache.flink.runtime.testingUtils.{ScalaTestingUtils, TestingUtils} -import org.apache.flink.test.util.ForkableFlinkMiniCluster +import org.apache.flink.runtime.testingUtils.{ScalaTestingUtils, TestingCluster, TestingUtils} import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner import org.scalatest.{BeforeAndAfterAll, Matchers, WordSpecLike} @@ -100,7 +98,7 @@ class TaskManagerFailsITCase(_system: ActorSystem) val jobGraph = new JobGraph("Pointwise Job", sender, receiver) val jobID = jobGraph.getJobID - val cluster = ForkableFlinkMiniCluster.startCluster(num_tasks, 2) + val cluster = TestingUtils.startTestingCluster(num_tasks, 2) val jmGateway = cluster.getLeaderGateway(TestingUtils.TESTING_DURATION) @@ -152,7 +150,7 @@ class TaskManagerFailsITCase(_system: ActorSystem) val jobGraph = new JobGraph("Pointwise Job", sender, receiver) val jobID = jobGraph.getJobID - val cluster = ForkableFlinkMiniCluster.startCluster(num_tasks, 2) + val cluster = TestingUtils.startTestingCluster(num_tasks, 2) val taskManagers = cluster.getTaskManagers val jmGateway = cluster.getLeaderGateway(TestingUtils.TESTING_DURATION) @@ -239,11 +237,11 @@ class TaskManagerFailsITCase(_system: ActorSystem) } } - def createDeathwatchCluster(numSlots: Int, numTaskmanagers: Int): ForkableFlinkMiniCluster = { + def createDeathwatchCluster(numSlots: Int, numTaskmanagers: Int): TestingCluster = { val config = new Configuration() config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlots) config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTaskmanagers) - new ForkableFlinkMiniCluster(config, singleActorSystem = false) + new TestingCluster(config, singleActorSystem = false) } } diff --git a/flink-yarn-tests/pom.xml b/flink-yarn-tests/pom.xml index 8c211ef281a49..ffdca36264b9d 100644 --- a/flink-yarn-tests/pom.xml +++ b/flink-yarn-tests/pom.xml @@ -48,6 +48,14 @@ under the License. test + + org.apache.flink + flink-runtime_2.10 + ${project.version} + test + test-jar + + org.apache.flink diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java index 02430127add5d..31a3d98832bb4 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java @@ -48,7 +48,6 @@ import org.junit.ClassRule; import org.junit.rules.TemporaryFolder; import org.mockito.Mockito; -import org.mockito.verification.VerificationMode; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.Marker; diff --git a/tools/maven/scalastyle-config.xml b/tools/maven/scalastyle-config.xml index f7bb0d4819ceb..0f7f6bbcb0484 100644 --- a/tools/maven/scalastyle-config.xml +++ b/tools/maven/scalastyle-config.xml @@ -86,7 +86,7 @@ - + From 29c38933f8eea252dbe45cd51de7981e670de433 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 1 Sep 2016 14:41:44 +0200 Subject: [PATCH 031/299] [FLINK-4456] Replace Akka specific types by interfaces in Task Introduce TaskExecutionStateListener for Task Replace JobManagerGateway in Task by InputSplitProvider and CheckpointNotifier Replace the TaskManager ActorGateway by TaskManagerConnection in Task Rename taskmanager.CheckpointNotifier into CheckpointResponder; rename TaskExecutionStateListener.notifyTaskExecutionState into notifyTaskExecutionStateChanged Remove InputSplitProvider.start; add ClassLoader parameter to InputSplitProvider.getNextInputSplit Removes the unused class InputSplitIterator. Update InputSplitProvider JavaDocs This closes #2456. --- .../ExecutionPlanAfterExecutionTest.java | 5 +- .../jobgraph/tasks/InputSplitIterator.java | 88 --------------- .../jobgraph/tasks/InputSplitProvider.java | 5 +- .../runtime/operators/DataSourceTask.java | 2 +- .../ActorGatewayCheckpointResponder.java | 78 +++++++++++++ ...ctorGatewayTaskExecutionStateListener.java | 42 +++++++ .../ActorGatewayTaskManagerConnection.java | 59 ++++++++++ .../taskmanager/CheckpointResponder.java | 63 +++++++++++ .../taskmanager/RuntimeEnvironment.java | 21 ++-- .../flink/runtime/taskmanager/Task.java | 105 ++++++++---------- .../TaskExecutionStateListener.java | 29 +++++ .../taskmanager/TaskInputSplitProvider.java | 55 ++++----- .../taskmanager/TaskManagerConnection.java | 57 ++++++++++ .../runtime/taskmanager/TaskManager.scala | 17 ++- .../testutils/MockInputSplitProvider.java | 3 +- .../taskmanager/TaskAsyncCallTest.java | 13 +-- .../TaskInputSplitProviderTest.java | 16 +-- .../runtime/taskmanager/TaskStopTest.java | 8 +- .../flink/runtime/taskmanager/TaskTest.java | 60 ++++++---- .../source/InputFormatSourceFunction.java | 2 +- .../source/InputFormatSourceFunctionTest.java | 5 +- .../tasks/InterruptSensitiveRestoreTest.java | 13 +-- .../runtime/tasks/StreamTaskTest.java | 75 ++++--------- 23 files changed, 519 insertions(+), 302 deletions(-) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitIterator.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayCheckpointResponder.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskExecutionStateListener.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskManagerConnection.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/CheckpointResponder.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionStateListener.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerConnection.java diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanAfterExecutionTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanAfterExecutionTest.java index 2bffba95e2b1e..4ec0e4790b775 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanAfterExecutionTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanAfterExecutionTest.java @@ -25,12 +25,15 @@ import org.apache.flink.api.java.aggregation.Aggregations; import org.apache.flink.api.java.io.DiscardingOutputFormat; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.util.TestLogger; import org.junit.Test; +import java.io.Serializable; + import static org.junit.Assert.fail; @SuppressWarnings("serial") -public class ExecutionPlanAfterExecutionTest implements java.io.Serializable { +public class ExecutionPlanAfterExecutionTest extends TestLogger implements Serializable { @Test public void testExecuteAfterGetExecutionPlan() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitIterator.java deleted file mode 100644 index a3d700a514c27..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitIterator.java +++ /dev/null @@ -1,88 +0,0 @@ -/* - * 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.runtime.jobgraph.tasks; - -import java.util.Iterator; - -import org.apache.flink.core.io.InputSplit; - -/** - * The input split iterator allows a task to iterate over all input splits it is supposed to - * consume. Internally, the input split iterator calls an {@link InputSplitProvider} on each next call in - * order to facilitate lazy split assignment. - * - * @param - */ -public class InputSplitIterator implements Iterator { - - /** - * The {@link InputSplitProvider} that is called to provide new input splits. - */ - private final InputSplitProvider inputSplitProvider; - - /** - * Buffers the next input split to be returned by this iterator or null it no split is buffered. - */ - private T nextInputSplit = null; - - /** - * Constructs a new input split iterator. - * - * @param inputSplitProvider - * the input split provider to be called for new input splits - */ - public InputSplitIterator(final InputSplitProvider inputSplitProvider) { - this.inputSplitProvider = inputSplitProvider; - } - - - @SuppressWarnings("unchecked") - @Override - public boolean hasNext() { - - if (this.nextInputSplit == null) { - this.nextInputSplit = (T) inputSplitProvider.getNextInputSplit(); - } - - return this.nextInputSplit != null; - } - - - @SuppressWarnings("unchecked") - @Override - public T next() { - T retVal = null; - - if (this.nextInputSplit == null) { - this.nextInputSplit = (T) inputSplitProvider.getNextInputSplit(); - } - - retVal = this.nextInputSplit; - this.nextInputSplit = null; - - return retVal; - } - - - @Override - public void remove() { - throw new RuntimeException("The InputSplitIterator does not implement the remove method"); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProvider.java index 5e7a40fb1c6a7..e0cde1778f193 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProvider.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProvider.java @@ -30,9 +30,10 @@ public interface InputSplitProvider { /** * Requests the next input split to be consumed by the calling task. - * + * + * @param userCodeClassLoader used to deserialize input splits * @return the next input split to be consumed by the calling task or null if the * task shall not consume any further input splits. */ - InputSplit getNextInputSplit(); + InputSplit getNextInputSplit(ClassLoader userCodeClassLoader); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java index 68e29b6ff6a7a..c062bf8ba8854 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java @@ -333,7 +333,7 @@ public boolean hasNext() { return true; } - InputSplit split = provider.getNextInputSplit(); + InputSplit split = provider.getNextInputSplit(getUserCodeClassLoader()); if (split != null) { this.nextSplit = split; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayCheckpointResponder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayCheckpointResponder.java new file mode 100644 index 0000000000000..56e5922e46362 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayCheckpointResponder.java @@ -0,0 +1,78 @@ +/* + * 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.runtime.taskmanager; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.instance.ActorGateway; +import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; +import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint; +import org.apache.flink.runtime.state.ChainedStateHandle; +import org.apache.flink.runtime.state.KeyGroupsStateHandle; +import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.util.Preconditions; + +import java.util.List; + +/** + * Implementation using {@link ActorGateway} to forward the messages. + */ +public class ActorGatewayCheckpointResponder implements CheckpointResponder { + + private final ActorGateway actorGateway; + + public ActorGatewayCheckpointResponder(ActorGateway actorGateway) { + this.actorGateway = Preconditions.checkNotNull(actorGateway); + } + + @Override + public void acknowledgeCheckpoint( + JobID jobID, + ExecutionAttemptID executionAttemptID, + long checkpointID, + ChainedStateHandle chainedStateHandle, + List keyGroupStateHandles) { + + AcknowledgeCheckpoint message = new AcknowledgeCheckpoint( + jobID, + executionAttemptID, + checkpointID, + chainedStateHandle, + keyGroupStateHandles); + + actorGateway.tell(message); + } + + @Override + public void declineCheckpoint( + JobID jobID, + ExecutionAttemptID executionAttemptID, + long checkpointID, + long checkpointTimestamp) { + + DeclineCheckpoint decline = new DeclineCheckpoint( + jobID, + executionAttemptID, + checkpointID, + checkpointTimestamp); + + actorGateway.tell(decline); + + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskExecutionStateListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskExecutionStateListener.java new file mode 100644 index 0000000000000..d729dbbf5ad3b --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskExecutionStateListener.java @@ -0,0 +1,42 @@ +/* + * 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.runtime.taskmanager; + +import org.apache.flink.runtime.instance.ActorGateway; +import org.apache.flink.runtime.messages.TaskMessages; +import org.apache.flink.util.Preconditions; + +/** + * Implementation using {@link ActorGateway} to forward the messages. + */ +public class ActorGatewayTaskExecutionStateListener implements TaskExecutionStateListener { + + private final ActorGateway actorGateway; + + public ActorGatewayTaskExecutionStateListener(ActorGateway actorGateway) { + this.actorGateway = Preconditions.checkNotNull(actorGateway); + } + + @Override + public void notifyTaskExecutionStateChanged(TaskExecutionState taskExecutionState) { + TaskMessages.UpdateTaskExecutionState actorMessage = new TaskMessages.UpdateTaskExecutionState(taskExecutionState); + + actorGateway.tell(actorMessage); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskManagerConnection.java new file mode 100644 index 0000000000000..cddac55da7ec2 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskManagerConnection.java @@ -0,0 +1,59 @@ +/* + * 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.runtime.taskmanager; + +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.instance.ActorGateway; +import org.apache.flink.runtime.messages.TaskManagerMessages; +import org.apache.flink.runtime.messages.TaskMessages; +import org.apache.flink.util.Preconditions; + +/** + * Implementation using {@link ActorGateway} to forward the messages. + */ +public class ActorGatewayTaskManagerConnection implements TaskManagerConnection { + + private final ActorGateway actorGateway; + + public ActorGatewayTaskManagerConnection(ActorGateway actorGateway) { + this.actorGateway = Preconditions.checkNotNull(actorGateway); + } + + @Override + public void notifyFinalState(ExecutionAttemptID executionAttemptID) { + actorGateway.tell(new TaskMessages.TaskInFinalState(executionAttemptID)); + } + + @Override + public void notifyFatalError(String message, Throwable cause) { + actorGateway.tell(new TaskManagerMessages.FatalError(message, cause)); + } + + @Override + public void failTask(ExecutionAttemptID executionAttemptID, Throwable cause) { + actorGateway.tell(new TaskMessages.FailTask(executionAttemptID, cause)); + } + + @Override + public void updateTaskExecutionState(TaskExecutionState taskExecutionState) { + TaskMessages.UpdateTaskExecutionState actorMessage = new TaskMessages.UpdateTaskExecutionState(taskExecutionState); + + actorGateway.tell(actorMessage); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/CheckpointResponder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/CheckpointResponder.java new file mode 100644 index 0000000000000..9d5c4e16b75ca --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/CheckpointResponder.java @@ -0,0 +1,63 @@ +/* + * 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.runtime.taskmanager; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.state.ChainedStateHandle; +import org.apache.flink.runtime.state.KeyGroupsStateHandle; +import org.apache.flink.runtime.state.StreamStateHandle; + +import java.util.List; + +/** + * Responder for checkpoint acknowledge and decline messages in the {@link Task}. + */ +public interface CheckpointResponder { + + /** + * Acknowledges the given checkpoint. + * + * @param jobID Job ID of the running job + * @param executionAttemptID Execution attempt ID of the running task + * @param checkpointID Checkpoint ID of the checkpoint + * @param chainedStateHandle Chained state handle + * @param keyGroupStateHandles State handles for key groups + */ + void acknowledgeCheckpoint( + JobID jobID, + ExecutionAttemptID executionAttemptID, + long checkpointID, + ChainedStateHandle chainedStateHandle, + List keyGroupStateHandles); + + /** + * Declines the given checkpoint. + * + * @param jobID Job ID of the running job + * @param executionAttemptID Execution attempt ID of the running task + * @param checkpointID Checkpoint ID of the checkpoint + * @param checkpointTimestamp Timestamp of the checkpoint + */ + void declineCheckpoint( + JobID jobID, + ExecutionAttemptID executionAttemptID, + long checkpointID, + long checkpointTimestamp); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java index d54826a3a19e1..3e4ba4df0022d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java @@ -27,14 +27,12 @@ import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; -import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; import org.apache.flink.runtime.memory.MemoryManager; -import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.query.TaskKvStateRegistry; import org.apache.flink.runtime.state.ChainedStateHandle; @@ -74,7 +72,7 @@ public class RuntimeEnvironment implements Environment { private final ResultPartitionWriter[] writers; private final InputGate[] inputGates; - private final ActorGateway jobManager; + private final CheckpointResponder checkpointResponder; private final AccumulatorRegistry accumulatorRegistry; @@ -105,7 +103,7 @@ public RuntimeEnvironment( Map> distCacheEntries, ResultPartitionWriter[] writers, InputGate[] inputGates, - ActorGateway jobManager, + CheckpointResponder checkpointResponder, TaskManagerRuntimeInfo taskManagerInfo, TaskMetricGroup metrics, Task containingTask) { @@ -127,7 +125,7 @@ public RuntimeEnvironment( this.distCacheEntries = checkNotNull(distCacheEntries); this.writers = checkNotNull(writers); this.inputGates = checkNotNull(inputGates); - this.jobManager = checkNotNull(jobManager); + this.checkpointResponder = checkNotNull(checkpointResponder); this.taskManagerInfo = checkNotNull(taskManagerInfo); this.containingTask = containingTask; this.metrics = metrics; @@ -251,14 +249,13 @@ public void acknowledgeCheckpoint( ChainedStateHandle chainedStateHandle, List keyGroupStateHandles) { - AcknowledgeCheckpoint message = new AcknowledgeCheckpoint( - jobId, - executionId, - checkpointId, - chainedStateHandle, - keyGroupStateHandles); - jobManager.tell(message); + checkpointResponder.acknowledgeCheckpoint( + jobId, + executionId, + checkpointId, + chainedStateHandle, + keyGroupStateHandles); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java index d09e03c7c8978..9994b7d8c5f6d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java @@ -26,6 +26,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.io.network.netty.PartitionStateChecker; +import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; @@ -40,7 +41,6 @@ import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.filecache.FileCache; -import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.network.NetworkEnvironment; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; @@ -54,11 +54,6 @@ import org.apache.flink.runtime.jobgraph.tasks.StatefulTask; import org.apache.flink.runtime.jobgraph.tasks.StoppableTask; import org.apache.flink.runtime.memory.MemoryManager; -import org.apache.flink.runtime.messages.TaskManagerMessages.FatalError; -import org.apache.flink.runtime.messages.TaskMessages.FailTask; -import org.apache.flink.runtime.messages.TaskMessages.TaskInFinalState; -import org.apache.flink.runtime.messages.TaskMessages.UpdateTaskExecutionState; -import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint; import org.apache.flink.runtime.query.TaskKvStateRegistry; import org.apache.flink.runtime.state.ChainedStateHandle; import org.apache.flink.runtime.state.KeyGroupsStateHandle; @@ -68,8 +63,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.concurrent.duration.FiniteDuration; - import java.io.IOException; import java.net.URL; import java.util.HashMap; @@ -174,17 +167,17 @@ public class Task implements Runnable { private final Map inputGatesById; - /** Gateway to the TaskManager that spawned this task */ - private final ActorGateway taskManager; + /** Connection to the task manager */ + private final TaskManagerConnection taskManagerConnection; - /** Gateway to the JobManager */ - private final ActorGateway jobManager; + /** Input split provider for the task */ + private final InputSplitProvider inputSplitProvider; - /** All actors that want to be notified about changes in the task's execution state */ - private final List executionListenerActors; + /** Checkpoint notifier used to communicate with the CheckpointCoordinator */ + private final CheckpointResponder checkpointResponder; - /** The timeout for all ask operations on actors */ - private final FiniteDuration actorAskTimeout; + /** All listener that want to be notified about changes in the task's execution state */ + private final List taskExecutionStateListeners; /** The library cache, from which the task can request its required JAR files */ private final LibraryCacheManager libraryCache; @@ -244,20 +237,21 @@ public class Task implements Runnable { *

IMPORTANT: This constructor may not start any work that would need to * be undone in the case of a failing task deployment.

*/ - public Task(TaskDeploymentDescriptor tdd, - MemoryManager memManager, - IOManager ioManager, - NetworkEnvironment networkEnvironment, - JobManagerCommunicationFactory jobManagerCommunicationFactory, - BroadcastVariableManager bcVarManager, - ActorGateway taskManagerActor, - ActorGateway jobManagerActor, - FiniteDuration actorAskTimeout, - LibraryCacheManager libraryCache, - FileCache fileCache, - TaskManagerRuntimeInfo taskManagerConfig, - TaskMetricGroup metricGroup) - { + public Task( + TaskDeploymentDescriptor tdd, + MemoryManager memManager, + IOManager ioManager, + NetworkEnvironment networkEnvironment, + JobManagerCommunicationFactory jobManagerCommunicationFactory, + BroadcastVariableManager bcVarManager, + TaskManagerConnection taskManagerConnection, + InputSplitProvider inputSplitProvider, + CheckpointResponder checkpointResponder, + LibraryCacheManager libraryCache, + FileCache fileCache, + TaskManagerRuntimeInfo taskManagerConfig, + TaskMetricGroup metricGroup) { + this.taskInfo = checkNotNull(tdd.getTaskInfo()); this.jobId = checkNotNull(tdd.getJobID()); this.vertexId = checkNotNull(tdd.getVertexID()); @@ -281,16 +275,16 @@ public Task(TaskDeploymentDescriptor tdd, this.broadcastVariableManager = checkNotNull(bcVarManager); this.accumulatorRegistry = new AccumulatorRegistry(jobId, executionId); - this.jobManager = checkNotNull(jobManagerActor); - this.taskManager = checkNotNull(taskManagerActor); - this.actorAskTimeout = checkNotNull(actorAskTimeout); + this.inputSplitProvider = checkNotNull(inputSplitProvider); + this.checkpointResponder = checkNotNull(checkpointResponder); + this.taskManagerConnection = checkNotNull(taskManagerConnection); this.libraryCache = checkNotNull(libraryCache); this.fileCache = checkNotNull(fileCache); this.network = checkNotNull(networkEnvironment); this.taskManagerConfig = checkNotNull(taskManagerConfig); - this.executionListenerActors = new CopyOnWriteArrayList(); + this.taskExecutionStateListeners = new CopyOnWriteArrayList<>(); this.metrics = metricGroup; // create the reader and writer structures @@ -539,19 +533,16 @@ else if (current == ExecutionState.CANCELING) { // call the user code initialization methods // ---------------------------------------------------------------- - TaskInputSplitProvider splitProvider = new TaskInputSplitProvider(jobManager, - jobId, vertexId, executionId, userCodeClassLoader, actorAskTimeout); - TaskKvStateRegistry kvStateRegistry = network .createKvStateTaskRegistry(jobId, getJobVertexId()); - Environment env = new RuntimeEnvironment(jobId, vertexId, executionId, - executionConfig, taskInfo, jobConfiguration, taskConfiguration, - userCodeClassLoader, memoryManager, ioManager, - broadcastVariableManager, accumulatorRegistry, - kvStateRegistry, - splitProvider, distributedCacheEntries, - writers, inputGates, jobManager, taskManagerConfig, metrics, this); + Environment env = new RuntimeEnvironment( + jobId, vertexId, executionId, executionConfig, taskInfo, + jobConfiguration, taskConfiguration, userCodeClassLoader, + memoryManager, ioManager, broadcastVariableManager, + accumulatorRegistry, kvStateRegistry, inputSplitProvider, + distributedCacheEntries, writers, inputGates, + checkpointResponder, taskManagerConfig, metrics, this); // let the task code create its readers and writers invokable.setEnvironment(env); @@ -588,11 +579,9 @@ else if (current == ExecutionState.CANCELING) { throw new CancelTaskException(); } - // notify everyone that we switched to running. especially the TaskManager needs - // to know this! + // notify everyone that we switched to running notifyObservers(ExecutionState.RUNNING, null); - taskManager.tell(new UpdateTaskExecutionState( - new TaskExecutionState(jobId, executionId, ExecutionState.RUNNING))); + taskManagerConnection.updateTaskExecutionState(new TaskExecutionState(jobId, executionId, ExecutionState.RUNNING)); // make sure the user code classloader is accessible thread-locally executingThread.setContextClassLoader(userCodeClassLoader); @@ -785,11 +774,11 @@ private void removeCachedFiles(Map> entries, FileCache file } private void notifyFinalState() { - taskManager.tell(new TaskInFinalState(executionId)); + taskManagerConnection.notifyFinalState(executionId); } private void notifyFatalError(String message, Throwable cause) { - taskManager.tell(new FatalError(message, cause)); + taskManagerConnection.notifyFatalError(message, cause); } // ---------------------------------------------------------------------------------------------------------------- @@ -815,7 +804,7 @@ public void run() { ((StoppableTask)Task.this.invokable).stop(); } catch(RuntimeException e) { LOG.error("Stopping task " + taskNameWithSubtask + " failed.", e); - taskManager.tell(new FailTask(executionId, e)); + taskManagerConnection.failTask(executionId, e); } } }; @@ -910,8 +899,8 @@ else if (current == ExecutionState.RUNNING) { // State Listeners // ------------------------------------------------------------------------ - public void registerExecutionListener(ActorGateway listener) { - executionListenerActors.add(listener); + public void registerExecutionListener(TaskExecutionStateListener listener) { + taskExecutionStateListeners.add(listener); } private void notifyObservers(ExecutionState newState, Throwable error) { @@ -923,10 +912,9 @@ private void notifyObservers(ExecutionState newState, Throwable error) { } TaskExecutionState stateUpdate = new TaskExecutionState(jobId, executionId, newState, error); - UpdateTaskExecutionState actorMessage = new UpdateTaskExecutionState(stateUpdate); - for (ActorGateway listener : executionListenerActors) { - listener.tell(actorMessage); + for (TaskExecutionStateListener listener : taskExecutionStateListeners) { + listener.notifyTaskExecutionStateChanged(stateUpdate); } } @@ -936,7 +924,7 @@ private void notifyObservers(ExecutionState newState, Throwable error) { /** * Calls the invokable to trigger a checkpoint, if the invokable implements the interface - * {@link org.apache.flink.runtime.jobgraph.tasks.StatefulTask}. + * {@link StatefulTask}. * * @param checkpointID The ID identifying the checkpoint. * @param checkpointTimestamp The timestamp associated with the checkpoint. @@ -957,8 +945,7 @@ public void run() { try { boolean success = statefulTask.triggerCheckpoint(checkpointID, checkpointTimestamp); if (!success) { - DeclineCheckpoint decline = new DeclineCheckpoint(jobId, getExecutionId(), checkpointID, checkpointTimestamp); - jobManager.tell(decline); + checkpointResponder.declineCheckpoint(jobId, getExecutionId(), checkpointID, checkpointTimestamp); } } catch (Throwable t) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionStateListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionStateListener.java new file mode 100644 index 0000000000000..9fa9c9025e5c3 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionStateListener.java @@ -0,0 +1,29 @@ +/* + * 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.runtime.taskmanager; + +public interface TaskExecutionStateListener { + + /** + * Called whenever the task's execution state changes + * + * @param taskExecutionState describing the task execution state change + */ + void notifyTaskExecutionStateChanged(TaskExecutionState taskExecutionState); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProvider.java index cb78c16c2c3a3..60beae097fe01 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProvider.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProvider.java @@ -27,55 +27,55 @@ import org.apache.flink.runtime.messages.JobManagerMessages; import org.apache.flink.util.InstantiationUtil; +import org.apache.flink.util.Preconditions; import scala.concurrent.Await; import scala.concurrent.Future; import scala.concurrent.duration.FiniteDuration; +/** + * Implementation using {@link ActorGateway} to forward the messages. + */ public class TaskInputSplitProvider implements InputSplitProvider { private final ActorGateway jobManager; - private final JobID jobId; + private final JobID jobID; - private final JobVertexID vertexId; + private final JobVertexID vertexID; private final ExecutionAttemptID executionID; - private final ClassLoader usercodeClassLoader; - private final FiniteDuration timeout; - + + public TaskInputSplitProvider( - ActorGateway jobManager, - JobID jobId, - JobVertexID vertexId, - ExecutionAttemptID executionID, - ClassLoader userCodeClassLoader, - FiniteDuration timeout) - { - this.jobManager = jobManager; - this.jobId = jobId; - this.vertexId = vertexId; - this.executionID = executionID; - this.usercodeClassLoader = userCodeClassLoader; - this.timeout = timeout; + ActorGateway jobManager, + JobID jobID, + JobVertexID vertexID, + ExecutionAttemptID executionID, + FiniteDuration timeout) { + + this.jobManager = Preconditions.checkNotNull(jobManager); + this.jobID = Preconditions.checkNotNull(jobID); + this.vertexID = Preconditions.checkNotNull(vertexID); + this.executionID = Preconditions.checkNotNull(executionID); + this.timeout = Preconditions.checkNotNull(timeout); } @Override - public InputSplit getNextInputSplit() { + public InputSplit getNextInputSplit(ClassLoader userCodeClassLoader) { + Preconditions.checkNotNull(userCodeClassLoader); + try { final Future response = jobManager.ask( - new JobManagerMessages.RequestNextInputSplit(jobId, vertexId, executionID), + new JobManagerMessages.RequestNextInputSplit(jobID, vertexID, executionID), timeout); final Object result = Await.result(response, timeout); - if(!(result instanceof JobManagerMessages.NextInputSplit)){ - throw new RuntimeException("RequestNextInputSplit requires a response of type " + - "NextInputSplit. Instead response is of type " + result.getClass() + "."); - } else { + if(result instanceof JobManagerMessages.NextInputSplit){ final JobManagerMessages.NextInputSplit nextInputSplit = - (JobManagerMessages.NextInputSplit) result; + (JobManagerMessages.NextInputSplit) result; byte[] serializedData = nextInputSplit.splitData(); @@ -83,9 +83,12 @@ public InputSplit getNextInputSplit() { return null; } else { Object deserialized = InstantiationUtil.deserializeObject(serializedData, - usercodeClassLoader); + userCodeClassLoader); return (InputSplit) deserialized; } + } else { + throw new Exception("RequestNextInputSplit requires a response of type " + + "NextInputSplit. Instead response is of type " + result.getClass() + '.'); } } catch (Exception e) { throw new RuntimeException("Requesting the next InputSplit failed.", e); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerConnection.java new file mode 100644 index 0000000000000..dc1b40f62d760 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerConnection.java @@ -0,0 +1,57 @@ +/* + * 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.runtime.taskmanager; + +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; + +/** + * Interface for the communication of the {@link Task} with the {@link TaskManager}. + */ +public interface TaskManagerConnection { + + /** + * Notifies the task manager that the given task is in a final state. + * + * @param executionAttemptID Execution attempt ID of the task + */ + void notifyFinalState(ExecutionAttemptID executionAttemptID); + + /** + * Notifies the task manager about a fatal error occurred in the task. + * + * @param message Message to report + * @param cause Cause of the fatal error + */ + void notifyFatalError(String message, Throwable cause); + + /** + * Tells the task manager to fail the given task. + * + * @param executionAttemptID Execution attempt ID of the task to fail + * @param cause Cause of the failure + */ + void failTask(ExecutionAttemptID executionAttemptID, Throwable cause); + + /** + * Notifies the task manager about the task execution state update. + * + * @param taskExecutionState Task execution state update + */ + void updateTaskExecutionState(TaskExecutionState taskExecutionState); +} diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index de85f30f35590..8ebdd803bd9af 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -1149,6 +1149,17 @@ class TaskManager( val taskMetricGroup = taskManagerMetricGroup.addTaskForJob(tdd) + val inputSplitProvider = new TaskInputSplitProvider( + jobManagerGateway, + tdd.getJobID, + tdd.getVertexID, + tdd.getExecutionId, + config.timeout) + + val checkpointResponder = new ActorGatewayCheckpointResponder(jobManagerGateway); + + val taskManagerConnection = new ActorGatewayTaskManagerConnection(selfGateway) + val task = new Task( tdd, memoryManager, @@ -1156,9 +1167,9 @@ class TaskManager( network, jmFactory, bcVarManager, - selfGateway, - jobManagerGateway, - config.timeout, + taskManagerConnection, + inputSplitProvider, + checkpointResponder, libCache, fileCache, runtimeInfo, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockInputSplitProvider.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockInputSplitProvider.java index 1d405f04e8b9c..a17484f78844a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockInputSplitProvider.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockInputSplitProvider.java @@ -84,7 +84,7 @@ public void addInputSplits(final String path, final int noSplits) { @Override - public InputSplit getNextInputSplit() { + public InputSplit getNextInputSplit(ClassLoader userCodeClassLoader) { if (this.nextSplit < this.inputSplits.length) { return this.inputSplits[this.nextSplit++]; @@ -92,5 +92,4 @@ public InputSplit getNextInputSplit() { return null; } - } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java index 9501c7c58e585..a5f401947a0fe 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; @@ -32,8 +33,6 @@ import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.filecache.FileCache; -import org.apache.flink.runtime.instance.ActorGateway; -import org.apache.flink.runtime.instance.DummyActorGateway; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.network.NetworkEnvironment; import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier; @@ -52,12 +51,9 @@ import org.junit.Before; import org.junit.Test; -import scala.concurrent.duration.FiniteDuration; - import java.net.URL; import java.util.Collections; import java.util.List; -import java.util.concurrent.TimeUnit; import static org.junit.Assert.assertFalse; import static org.junit.Assert.fail; @@ -170,16 +166,15 @@ private static Task createTask() throws Exception { Collections.emptyList(), 0); - ActorGateway taskManagerGateway = DummyActorGateway.INSTANCE; return new Task(tdd, mock(MemoryManager.class), mock(IOManager.class), networkEnvironment, jobManagerCommunicationFactory, mock(BroadcastVariableManager.class), - taskManagerGateway, - DummyActorGateway.INSTANCE, - new FiniteDuration(60, TimeUnit.SECONDS), + mock(TaskManagerConnection.class), + mock(InputSplitProvider.class), + mock(CheckpointResponder.class), libCache, mock(FileCache.class), new TaskManagerRuntimeInfo("localhost", new Configuration(), System.getProperty("java.io.tmpdir")), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProviderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProviderTest.java index 4ccce1d9bc9bd..642300d439506 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProviderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProviderTest.java @@ -47,22 +47,22 @@ public void testRequestNextInputSplitWithInvalidExecutionID() { final TaskInputSplitProvider provider = new TaskInputSplitProvider( - gateway, - jobID, - vertexID, - executionID, - getClass().getClassLoader(), - timeout - ); + gateway, + jobID, + vertexID, + executionID, + timeout); // The jobManager will return a - InputSplit nextInputSplit = provider.getNextInputSplit(); + InputSplit nextInputSplit = provider.getNextInputSplit(getClass().getClassLoader()); assertTrue(nextInputSplit == null); } public static class NullInputSplitGateway extends BaseTestingActorGateway { + private static final long serialVersionUID = -7733997150554492926L; + public NullInputSplitGateway() { super(TestingUtils.defaultExecutionContext()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java index cfa7fb663a2bd..d041465e9a962 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.TaskInfo; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; @@ -27,7 +28,6 @@ import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.filecache.FileCache; -import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.network.NetworkEnvironment; import org.apache.flink.runtime.jobgraph.JobVertexID; @@ -73,9 +73,9 @@ public void doMocking(AbstractInvokable taskMock) throws Exception { mock(NetworkEnvironment.class), mock(JobManagerCommunicationFactory.class), mock(BroadcastVariableManager.class), - mock(ActorGateway.class), - mock(ActorGateway.class), - mock(FiniteDuration.class), + mock(TaskManagerConnection.class), + mock(InputSplitProvider.class), + mock(CheckpointResponder.class), mock(LibraryCacheManager.class), mock(FileCache.class), mock(TaskManagerRuntimeInfo.class), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java index 9e8f8f85efe42..e5fdf32e59f4a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java @@ -42,6 +42,7 @@ import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.messages.TaskMessages; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; @@ -93,6 +94,9 @@ public class TaskTest { private ActorGateway jobManagerGateway; private ActorGateway listenerGateway; + private ActorGatewayTaskExecutionStateListener listener; + private ActorGatewayTaskManagerConnection taskManagerConnection; + private BlockingQueue taskManagerMessages; private BlockingQueue jobManagerMessages; private BlockingQueue listenerMessages; @@ -105,6 +109,9 @@ public void createQueuesAndActors() { taskManagerGateway = new ForwardingActorGateway(taskManagerMessages); jobManagerGateway = new ForwardingActorGateway(jobManagerMessages); listenerGateway = new ForwardingActorGateway(listenerMessages); + + listener = new ActorGatewayTaskExecutionStateListener(listenerGateway); + taskManagerConnection = new ActorGatewayTaskManagerConnection(taskManagerGateway); awaitLatch = new OneShotLatch(); triggerLatch = new OneShotLatch(); @@ -135,7 +142,7 @@ public void testRegularExecution() { assertFalse(task.isCanceledOrFailed()); assertNull(task.getFailureCause()); - task.registerExecutionListener(listenerGateway); + task.registerExecutionListener(listener); // go into the run method. we should switch to DEPLOYING, RUNNING, then // FINISHED, and all should be good @@ -210,7 +217,7 @@ public void testLibraryCacheRegistrationFailed() { assertFalse(task.isCanceledOrFailed()); assertNull(task.getFailureCause()); - task.registerExecutionListener(listenerGateway); + task.registerExecutionListener(listener); // should fail task.run(); @@ -253,7 +260,7 @@ public void testExecutionFailsInNetworkRegistration() { Task task = createTask(TestInvokableCorrect.class, libCache, network, jobManagerCommunicationFactory); - task.registerExecutionListener(listenerGateway); + task.registerExecutionListener(listener); task.run(); @@ -274,7 +281,7 @@ public void testExecutionFailsInNetworkRegistration() { public void testInvokableInstantiationFailed() { try { Task task = createTask(InvokableNonInstantiable.class); - task.registerExecutionListener(listenerGateway); + task.registerExecutionListener(listener); task.run(); @@ -295,7 +302,7 @@ public void testInvokableInstantiationFailed() { public void testExecutionFailsInInvoke() { try { Task task = createTask(InvokableWithExceptionInInvoke.class); - task.registerExecutionListener(listenerGateway); + task.registerExecutionListener(listener); task.run(); @@ -319,7 +326,7 @@ public void testExecutionFailsInInvoke() { public void testCancelDuringInvoke() { try { Task task = createTask(InvokableBlockingInInvoke.class); - task.registerExecutionListener(listenerGateway); + task.registerExecutionListener(listener); // run the task asynchronous task.startTaskThread(); @@ -353,7 +360,7 @@ public void testCancelDuringInvoke() { public void testFailExternallyDuringInvoke() { try { Task task = createTask(InvokableBlockingInInvoke.class); - task.registerExecutionListener(listenerGateway); + task.registerExecutionListener(listener); // run the task asynchronous task.startTaskThread(); @@ -386,7 +393,7 @@ public void testFailExternallyDuringInvoke() { public void testCanceledAfterExecutionFailedInInvoke() { try { Task task = createTask(InvokableWithExceptionInInvoke.class); - task.registerExecutionListener(listenerGateway); + task.registerExecutionListener(listener); task.run(); @@ -413,7 +420,7 @@ public void testCanceledAfterExecutionFailedInInvoke() { public void testExecutionFailesAfterCanceling() { try { Task task = createTask(InvokableWithExceptionOnTrigger.class); - task.registerExecutionListener(listenerGateway); + task.registerExecutionListener(listener); // run the task asynchronous task.startTaskThread(); @@ -450,7 +457,7 @@ public void testExecutionFailesAfterCanceling() { public void testExecutionFailsAfterTaskMarkedFailed() { try { Task task = createTask(InvokableWithExceptionOnTrigger.class); - task.registerExecutionListener(listenerGateway); + task.registerExecutionListener(listener); // run the task asynchronous task.startTaskThread(); @@ -618,21 +625,30 @@ private Task createTask( JobManagerCommunicationFactory jobManagerCommunicationFactory) { TaskDeploymentDescriptor tdd = createTaskDeploymentDescriptor(invokable); + + InputSplitProvider inputSplitProvider = new TaskInputSplitProvider( + jobManagerGateway, + tdd.getJobID(), + tdd.getVertexID(), + tdd.getExecutionId(), + new FiniteDuration(60, TimeUnit.SECONDS)); + + CheckpointResponder checkpointResponder = new ActorGatewayCheckpointResponder(jobManagerGateway); return new Task( - tdd, - mock(MemoryManager.class), - mock(IOManager.class), - networkEnvironment, + tdd, + mock(MemoryManager.class), + mock(IOManager.class), + networkEnvironment, jobManagerCommunicationFactory, - mock(BroadcastVariableManager.class), - taskManagerGateway, - jobManagerGateway, - new FiniteDuration(60, TimeUnit.SECONDS), - libCache, - mock(FileCache.class), - new TaskManagerRuntimeInfo("localhost", new Configuration(), System.getProperty("java.io.tmpdir")), - mock(TaskMetricGroup.class)); + mock(BroadcastVariableManager.class), + taskManagerConnection, + inputSplitProvider, + checkpointResponder, + libCache, + mock(FileCache.class), + new TaskManagerRuntimeInfo("localhost", new Configuration(), System.getProperty("java.io.tmpdir")), + mock(TaskMetricGroup.class)); } private TaskDeploymentDescriptor createTaskDeploymentDescriptor(Class invokable) { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunction.java index e3e5c541a1ccb..343affe13f604 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunction.java @@ -146,7 +146,7 @@ public boolean hasNext() { return true; } - InputSplit split = provider.getNextInputSplit(); + InputSplit split = provider.getNextInputSplit(getRuntimeContext().getUserCodeClassLoader()); if (split != null) { this.nextSplit = split; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunctionTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunctionTest.java index a41c7dbfe906d..d1131b40fa90e 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunctionTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunctionTest.java @@ -84,6 +84,7 @@ private void testFormatLifecycle(final boolean midCancel) throws Exception { private static class LifeCycleTestInputFormat extends RichInputFormat { + private static final long serialVersionUID = 7408902249499583273L; private boolean isConfigured = false; private boolean isInputFormatOpen = false; private boolean isSplitOpen = false; @@ -128,6 +129,8 @@ public InputSplit[] createInputSplits(int minNumSplits) throws IOException { for (int i = 0; i < minNumSplits; i++) { final int idx = i; splits[idx] = new InputSplit() { + private static final long serialVersionUID = -1480792932361908285L; + @Override public int getSplitNumber() { return idx; @@ -270,7 +273,7 @@ public InputSplitProvider getInputSplitProvider() { return new InputSplitProvider() { @Override - public InputSplit getNextInputSplit() { + public InputSplit getNextInputSplit(ClassLoader userCodeClassLoader) { if (nextSplit < inputSplits.length) { return inputSplits[nextSplit++]; } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java index e1c9407bdf970..47f1bd541f648 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java @@ -32,10 +32,10 @@ import org.apache.flink.runtime.execution.librarycache.FallbackLibraryCacheManager; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.filecache.FileCache; -import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.network.NetworkEnvironment; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.operators.testutils.UnregisteredTaskMetricsGroup; import org.apache.flink.runtime.query.TaskKvStateRegistry; @@ -44,7 +44,9 @@ import org.apache.flink.runtime.taskmanager.JobManagerCommunicationFactory; import org.apache.flink.runtime.state.KeyGroupsStateHandle; import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.runtime.taskmanager.CheckpointResponder; import org.apache.flink.runtime.taskmanager.Task; +import org.apache.flink.runtime.taskmanager.TaskManagerConnection; import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; import org.apache.flink.runtime.util.EnvironmentInformation; import org.apache.flink.streaming.api.TimeCharacteristic; @@ -56,15 +58,12 @@ import org.junit.Test; -import scala.concurrent.duration.FiniteDuration; - import java.io.EOFException; import java.io.IOException; import java.io.Serializable; import java.net.URL; import java.util.Collections; import java.util.List; -import java.util.concurrent.TimeUnit; import static org.junit.Assert.*; import static org.mockito.Mockito.*; @@ -153,9 +152,9 @@ private static Task createTask(TaskDeploymentDescriptor tdd) throws IOException networkEnvironment, mock(JobManagerCommunicationFactory.class), mock(BroadcastVariableManager.class), - mock(ActorGateway.class), - mock(ActorGateway.class), - new FiniteDuration(10, TimeUnit.SECONDS), + mock(TaskManagerConnection.class), + mock(InputSplitProvider.class), + mock(CheckpointResponder.class), new FallbackLibraryCacheManager(), new FileCache(new Configuration()), new TaskManagerRuntimeInfo( diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java index 0a9d2faa900f9..0c79c4e6f6607 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java @@ -41,12 +41,16 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; -import org.apache.flink.runtime.messages.TaskMessages; import org.apache.flink.runtime.query.TaskKvStateRegistry; +import org.apache.flink.runtime.taskmanager.CheckpointResponder; import org.apache.flink.runtime.taskmanager.JobManagerCommunicationFactory; import org.apache.flink.runtime.taskmanager.Task; +import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.runtime.taskmanager.TaskExecutionStateListener; +import org.apache.flink.runtime.taskmanager.TaskManagerConnection; import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.functions.source.SourceFunction; @@ -98,12 +102,12 @@ public void testEarlyCanceling() throws Exception { Task task = createTask(SourceStreamTask.class, cfg); - ExecutionStateListener executionStateListener = new ExecutionStateListener(); + TestingExecutionStateListener testingExecutionStateListener = new TestingExecutionStateListener(); - task.registerExecutionListener(executionStateListener); + task.registerExecutionListener(testingExecutionStateListener); task.startTaskThread(); - Future running = executionStateListener.notifyWhenExecutionState(ExecutionState.RUNNING); + Future running = testingExecutionStateListener.notifyWhenExecutionState(ExecutionState.RUNNING); // wait until the task thread reached state RUNNING ExecutionState executionState = Await.result(running, deadline.timeLeft()); @@ -118,7 +122,7 @@ public void testEarlyCanceling() throws Exception { // hit the task before the operator is deserialized task.cancelExecution(); - Future canceling = executionStateListener.notifyWhenExecutionState(ExecutionState.CANCELING); + Future canceling = testingExecutionStateListener.notifyWhenExecutionState(ExecutionState.CANCELING); executionState = Await.result(canceling, deadline.timeLeft()); @@ -137,9 +141,7 @@ public void testEarlyCanceling() throws Exception { // Test Utilities // ------------------------------------------------------------------------ - private static class ExecutionStateListener implements ActorGateway { - - private static final long serialVersionUID = 8926442805035692182L; + private static class TestingExecutionStateListener implements TaskExecutionStateListener { ExecutionState executionState = null; @@ -167,56 +169,17 @@ public Future notifyWhenExecutionState(ExecutionState executionS } @Override - public Future ask(Object message, FiniteDuration timeout) { - return null; - } - - @Override - public void tell(Object message) { - this.tell(message, null); - } - - @Override - public void tell(Object message, ActorGateway sender) { - if (message instanceof TaskMessages.UpdateTaskExecutionState) { - TaskMessages.UpdateTaskExecutionState updateTaskExecutionState = (TaskMessages.UpdateTaskExecutionState) message; - - synchronized (priorityQueue) { - this.executionState = updateTaskExecutionState.taskExecutionState().getExecutionState(); + public void notifyTaskExecutionStateChanged(TaskExecutionState taskExecutionState) { + synchronized (priorityQueue) { + this.executionState = taskExecutionState.getExecutionState(); - while (!priorityQueue.isEmpty() && priorityQueue.peek().f0.ordinal() <= this.executionState.ordinal()) { - Promise promise = priorityQueue.poll().f1; + while (!priorityQueue.isEmpty() && priorityQueue.peek().f0.ordinal() <= executionState.ordinal()) { + Promise promise = priorityQueue.poll().f1; - promise.success(this.executionState); - } + promise.success(executionState); } } } - - @Override - public void forward(Object message, ActorGateway sender) { - - } - - @Override - public Future retry(Object message, int numberRetries, FiniteDuration timeout, ExecutionContext executionContext) { - return null; - } - - @Override - public String path() { - return null; - } - - @Override - public ActorRef actor() { - return null; - } - - @Override - public UUID leaderSessionID() { - return null; - } } private Task createTask(Class invokable, StreamConfig taskConfig) throws Exception { @@ -254,9 +217,9 @@ private Task createTask(Class invokable, StreamConf network, jobManagerCommunicationFactory, mock(BroadcastVariableManager.class), - new DummyGateway(), - new DummyGateway(), - new FiniteDuration(60, TimeUnit.SECONDS), + mock(TaskManagerConnection.class), + mock(InputSplitProvider.class), + mock(CheckpointResponder.class), libCache, mock(FileCache.class), new TaskManagerRuntimeInfo("localhost", new Configuration(), System.getProperty("java.io.tmpdir")), From 28ac36657f5a55faab3f3944cc5510cc0d747967 Mon Sep 17 00:00:00 2001 From: Greg Hogan Date: Fri, 2 Sep 2016 10:42:30 -0400 Subject: [PATCH 032/299] [FLINK-4522] [docs] Gelly link broken in homepage The Gelly documentation was recently split into multiple pages in FLINK-4104 but was missing a redirect. This commit updates the Gelly redirect to point to the old page. This closes #2464 --- docs/redirects/gelly.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/redirects/gelly.md b/docs/redirects/gelly.md index 8d7d85031afa2..c61107b9d93b7 100644 --- a/docs/redirects/gelly.md +++ b/docs/redirects/gelly.md @@ -2,7 +2,7 @@ title: "Gelly" layout: redirect redirect: /dev/libs/gelly/index.html -permalink: /apis/batch/libs/gelly/index.html +permalink: /apis/batch/libs/gelly.html --- + +This document briefly describes how Flink security works in the context of various deployment mechanism (Standalone/Cluster vs YARN) +and the connectors that participates in Flink Job execution stage. This documentation can be helpful for both administrators and developers +who plans to run Flink on a secure environment. + +## Objective + +The primary goal of Flink security model is to enable secure data access for jobs within a cluster via connectors. In a production deployment scenario, +streaming jobs are understood to run for longer period of time (days/weeks/months) and the system must be able to authenticate against secure +data sources throughout the life of the job. The current implementation supports running Flink clusters (Job Manager/Task Manager/Jobs) under the +context of a Kerberos identity based on Keytab credential supplied during deployment time. Any jobs submitted will continue to run in the identity of the cluster. + +## How Flink Security works +Flink deployment includes running Job Manager/ZooKeeper, Task Manager(s), Web UI and Job(s). Jobs (user code) can be submitted through web UI and/or CLI. +A Job program may use one or more connectors (Kafka, HDFS, Cassandra, Flume, Kinesis etc.,) and each connector may have a specific security +requirements (Kerberos, database based, SSL/TLS, custom etc.,). While satisfying the security requirements for all the connectors evolves over a period +of time, at this time of writing, the following connectors/services are tested for Kerberos/Keytab based security. + +- Kafka (0.9) +- HDFS +- ZooKeeper + +Hadoop uses the UserGroupInformation (UGI) class to manage security. UGI is a static implementation that takes care of handling Kerberos authentication. The Flink bootstrap implementation +(JM/TM/CLI) takes care of instantiating UGI with the appropriate security credentials to establish the necessary security context. + +Services like Kafka and ZooKeeper use SASL/JAAS based authentication mechanism to authenticate against a Kerberos server. It expects JAAS configuration with a platform-specific login +module *name* to be provided. Managing per-connector configuration files will be an overhead and to overcome this requirement, a process-wide JAAS configuration object is +instantiated which serves standard ApplicationConfigurationEntry for the connectors that authenticates using SASL/JAAS mechanism. + +It is important to understand that the Flink processes (JM/TM/UI/Jobs) itself uses UGI's doAS() implementation to run under a specific user context, i.e. if Hadoop security is enabled +then the Flink processes will be running under a secure user account or else it will run as the OS login user account who starts the Flink cluster. + +## Security Configurations + +Secure credentials can be supplied by adding below configuration elements to Flink configuration file: + +- `security.keytab`: Absolute path to Kerberos keytab file that contains the user credentials/secret. + +- `security.principal`: User principal name that the Flink cluster should run as. + +The delegation token mechanism (*kinit cache*) is still supported for backward compatibility but enabling security using *keytab* configuration is the preferred and recommended approach. + +## Standalone Mode: + +Steps to run a secure Flink cluster in standalone/cluster mode: +- Add security configurations to Flink configuration file (on all cluster nodes) +- Make sure the Keytab file exist in the path as indicated in *security.keytab* configuration on all cluster nodes +- Deploy Flink cluster using cluster start/stop scripts or CLI + +## Yarn Mode: + +Steps to run secure Flink cluster in Yarn mode: +- Add security configurations to Flink configuration file (on the node from where cluster will be provisioned using Flink/Yarn CLI) +- Make sure the Keytab file exist in the path as indicated in *security.keytab* configuration +- Deploy Flink cluster using CLI + +In Yarn mode, the user supplied keytab will be copied over to the Yarn containers (App Master/JM and TM) as the Yarn local resource file. +Security implementation details are based on Yarn security + +## Token Renewal + +UGI and Kafka/ZK login module implementations takes care of auto-renewing the tickets upon reaching expiry and no further action is needed on the part of Flink. \ No newline at end of file diff --git a/docs/setup/config.md b/docs/setup/config.md index 51475ccadb638..54ef39403303d 100644 --- a/docs/setup/config.md +++ b/docs/setup/config.md @@ -95,18 +95,35 @@ These options are useful for debugging a Flink application for memory and garbag ### Kerberos -Flink supports Kerberos authentication of Hadoop services such as HDFS, YARN, or HBase. +Flink supports Kerberos authentication for the following services + ++ Hadoop Components: such as HDFS, YARN, or HBase. ++ Kafka Connectors (version 0.9+) ++ Zookeeper Server/Client + +Hadoop components relies on the UserGroupInformation (UGI) implementation to handle Kerberos authentication, whereas Kafka and Zookeeper services handles Kerberos authentication through SASL/JAAS implementation. **Kerberos is only properly supported in Hadoop version 2.6.1 and above. All other versions have critical bugs which might fail the Flink job unexpectedly.** +**Ticket cache** and **Keytab** modes are supported for all above mentioned services. + +> Ticket cache (Supported only to provide backward compatibility support. Keytab is the preferred approach for long running jobs) + While Hadoop uses Kerberos tickets to authenticate users with services initially, the authentication process continues differently afterwards. Instead of saving the ticket to authenticate on a later access, Hadoop creates its own security tokens (DelegationToken) that it passes around. These are authenticated to Kerberos periodically but are independent of the token renewal time. The tokens have a maximum life span identical to the Kerberos ticket maximum life span. -Please make sure to set the maximum ticket life span high long running jobs. The renewal time of the ticket, on the other hand, is not important because Hadoop abstracts this away using its own security tocken renewal system. Hadoop makes sure that tickets are renewed in time and you can be sure to be authenticated until the end of the ticket life time. +While using ticket cache mode, please make sure to set the maximum ticket life span high long running jobs. If you are on YARN, then it is sufficient to authenticate the client with Kerberos. On a Flink standalone cluster you need to ensure that, initially, all nodes are authenticated with Kerberos using the `kinit` tool. +> Keytab (security principal and keytab can be configured through Flink configuration file) +- `security.keytab`: Path to Keytab file +- `security.principal`: Principal associated with the keytab + +Kerberos ticket renewal is abstracted and automatically handled by the Hadoop/Kafka/ZK login modules and ensures that tickets are renewed in time and you can be sure to be authenticated until the end of the ticket life time. + +For Kafka and ZK, process-wide JAAS config will be created using the provided security credentials and the Kerberos authentication will be handled by Kafka/ZK login handlers. ### Other @@ -315,6 +332,12 @@ Previously this key was named `recovery.mode` and the default value was `standal - `high-availability.job.delay`: (Default `akka.ask.timeout`) Defines the delay before persisted jobs are recovered in case of a master recovery situation. Previously this key was named `recovery.job.delay`. +### ZooKeeper-Security + +- `zookeeper.sasl.disable`: (Default: `true`) Defines if SASL based authentication needs to be enabled or disabled. The configuration value can be set to "true" if ZooKeeper cluster is running in secure mode (Kerberos) + +- `zookeeper.sasl.service-name`: (Default: `zookeeper`) If the ZooKeeper server is configured with a different service name (default:"zookeeper") then it can be supplied using this configuration. A mismatch in service name between client and server configuration will cause the authentication to fail. + ## Environment - `env.log.dir`: (Defaults to the `log` directory under Flink's home) Defines the directory where the Flink logs are saved. It has to be an absolute path. diff --git a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java index c7fb6471eb6e3..575ffad79e46d 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java +++ b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java @@ -66,7 +66,7 @@ import org.apache.flink.runtime.messages.JobManagerMessages.StoppingFailure; import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepoint; import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepointSuccess; -import org.apache.flink.runtime.security.SecurityUtils; +import org.apache.flink.runtime.security.SecurityContext; import org.apache.flink.runtime.util.EnvironmentInformation; import org.apache.flink.util.Preconditions; import org.apache.flink.util.StringUtils; @@ -161,6 +161,9 @@ public CliFrontend(String configDir) throws Exception { "filesystem scheme from configuration.", e); } + this.config.setString(ConfigConstants.FLINK_BASE_DIR_PATH_KEY, configDirectory.getAbsolutePath() + + ".." + File.separator); + this.clientTimeout = AkkaUtils.getClientTimeout(config); } @@ -982,25 +985,7 @@ public int parseParameters(String[] args) { // do action switch (action) { case ACTION_RUN: - // run() needs to run in a secured environment for the optimizer. - if (SecurityUtils.isSecurityEnabled()) { - String message = "Secure Hadoop environment setup detected. Running in secure context."; - LOG.info(message); - - try { - return SecurityUtils.runSecured(new SecurityUtils.FlinkSecuredRunner() { - @Override - public Integer run() throws Exception { - return CliFrontend.this.run(params); - } - }); - } - catch (Exception e) { - return handleError(e); - } - } else { - return run(params); - } + return CliFrontend.this.run(params); case ACTION_LIST: return list(params); case ACTION_INFO: @@ -1037,12 +1022,19 @@ public Integer run() throws Exception { /** * Submits the job based on the arguments */ - public static void main(String[] args) { + public static void main(final String[] args) { EnvironmentInformation.logEnvironmentInfo(LOG, "Command Line Client", args); try { - CliFrontend cli = new CliFrontend(); - int retCode = cli.parseParameters(args); + final CliFrontend cli = new CliFrontend(); + SecurityContext.install(new SecurityContext.SecurityConfiguration().setFlinkConfiguration(cli.config)); + int retCode = SecurityContext.getInstalled() + .runSecured(new SecurityContext.FlinkSecuredRunner() { + @Override + public Integer run() { + return cli.parseParameters(args); + } + }); System.exit(retCode); } catch (Throwable t) { diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java index f0f1b6bdbba67..9e66e2aa74137 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java @@ -758,6 +758,12 @@ public final class ConfigConstants { @PublicEvolving public static final String HA_ZOOKEEPER_MAX_RETRY_ATTEMPTS = "high-availability.zookeeper.client.max-retry-attempts"; + @PublicEvolving + public static final String ZOOKEEPER_SASL_DISABLE = "zookeeper.sasl.disable"; + + @PublicEvolving + public static final String ZOOKEEPER_SASL_SERVICE_NAME = "zookeeper.sasl.service-name"; + /** Deprecated in favour of {@link #HA_ZOOKEEPER_QUORUM_KEY}. */ @Deprecated public static final String ZOOKEEPER_QUORUM_KEY = "recovery.zookeeper.quorum"; @@ -1233,6 +1239,9 @@ public final class ConfigConstants { /** ZooKeeper default leader port. */ public static final int DEFAULT_ZOOKEEPER_LEADER_PORT = 3888; + /** Defaults for ZK client security **/ + public static final boolean DEFAULT_ZOOKEEPER_SASL_DISABLE = true; + // ------------------------- Queryable state ------------------------------ /** Port to bind KvState server to. */ @@ -1279,6 +1288,19 @@ public final class ConfigConstants { /** The environment variable name which contains the location of the lib folder */ public static final String ENV_FLINK_LIB_DIR = "FLINK_LIB_DIR"; + // -------------------------------- Security ------------------------------- + + /** + * The config parameter defining security credentials required + * for securing Flink cluster. + */ + + /** Keytab file key name to be used in flink configuration file */ + public static final String SECURITY_KEYTAB_KEY = "security.keytab"; + + /** Kerberos security principal key name to be used in flink configuration file */ + public static final String SECURITY_PRINCIPAL_KEY = "security.principal"; + /** * Not instantiable. diff --git a/flink-core/src/main/java/org/apache/flink/util/Preconditions.java b/flink-core/src/main/java/org/apache/flink/util/Preconditions.java index ea6b9dd719e5d..e970c13ad640b 100644 --- a/flink-core/src/main/java/org/apache/flink/util/Preconditions.java +++ b/flink-core/src/main/java/org/apache/flink/util/Preconditions.java @@ -293,7 +293,7 @@ private static String format(@Nullable String template, @Nullable Object... args return builder.toString(); } - + // ------------------------------------------------------------------------ /** Private constructor to prevent instantiation */ diff --git a/flink-dist/src/main/flink-bin/conf/flink-jaas.conf b/flink-dist/src/main/flink-bin/conf/flink-jaas.conf new file mode 100644 index 0000000000000..d476e24151b83 --- /dev/null +++ b/flink-dist/src/main/flink-bin/conf/flink-jaas.conf @@ -0,0 +1,26 @@ +################################################################################ +# 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. +################################################################################ +# We are using this file as an workaround for the Kafka and ZK SASL implementation +# since they explicitly look for java.security.auth.login.config property +# The file itself is not used by the application since the internal implementation +# uses a process-wide in-memory java security configuration object. +# Please do not edit/delete this file - See FLINK-3929 +sample { + useKeyTab=false + useTicketCache=true; +}; \ No newline at end of file diff --git a/flink-dist/src/main/resources/flink-conf.yaml b/flink-dist/src/main/resources/flink-conf.yaml index 27fd84a560e6e..c8769221087da 100644 --- a/flink-dist/src/main/resources/flink-conf.yaml +++ b/flink-dist/src/main/resources/flink-conf.yaml @@ -139,6 +139,31 @@ jobmanager.web.port: 8081 # # "host1:clientPort,host2:clientPort,..." (default clientPort: 2181) # + # high-availability: zookeeper # high-availability.zookeeper.quorum: localhost:2181 # high-availability.zookeeper.storageDir: hdfs:///flink/ha/ + +#============================================================================== +# Flink Cluster Security Configuration (optional configuration) +#============================================================================== + +# Kerberos security for the connectors can be enabled by providing below configurations +# Security works in two modes - keytab/principal combination or using the Kerberos token cache +# If keytab and principal are not provided, token cache (manual kinit) will be used + +#security.keytab: /path/to/kerberos/keytab +#security.principal: flink-user + +#============================================================================== +# ZK Security Configuration (optional configuration) +#============================================================================== +# Below configurations are applicable if ZK quorum is configured for Kerberos security + +# SASL authentication is disabled by default and can be enabled by changig the value to false +# +# zookeeper.sasl.disable: true + +# Override below configuration to provide custom ZK service name if configured +# +# zookeeper.sasl.service-name: zookeeper \ No newline at end of file diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java index 8fb6af4f6783a..5ec39c2ab8d0d 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java @@ -582,10 +582,11 @@ public static Protos.TaskInfo.Builder createTaskManagerContext( // build the launch command boolean hasLogback = new File(workingDirectory, "logback.xml").exists(); boolean hasLog4j = new File(workingDirectory, "log4j.properties").exists(); + boolean hasKrb5 = false; String launchCommand = BootstrapTools.getTaskManagerShellCommand( flinkConfig, tmParams.containeredParameters(), ".", ".", - hasLogback, hasLog4j, taskManagerMainClass); + hasLogback, hasLog4j, hasKrb5, taskManagerMainClass); cmd.setValue(launchCommand); // build the environment variables diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java index c9748cb108b14..d844f5db01523 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java @@ -308,6 +308,7 @@ public static String getTaskManagerShellCommand( String logDirectory, boolean hasLogback, boolean hasLog4j, + boolean hasKrb5, Class mainClass) { StringBuilder tmCommand = new StringBuilder("$JAVA_HOME/bin/java"); @@ -328,6 +329,12 @@ public static String getTaskManagerShellCommand( tmCommand.append(" -Dlog4j.configuration=file:") .append(configDirectory).append("/log4j.properties"); } + + //applicable only for YarnMiniCluster secure test run + //krb5.conf file will be available as local resource in JM/TM container + if(hasKrb5) { + tmCommand.append(" -Djava.security.krb5.conf=krb5.conf"); + } } tmCommand.append(' ').append(mainClass.getName()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/security/JaasConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/security/JaasConfiguration.java new file mode 100644 index 0000000000000..c4527dd1498b0 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/security/JaasConfiguration.java @@ -0,0 +1,160 @@ +/* + * 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.runtime.security; + +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.annotation.Internal; +import org.apache.hadoop.security.authentication.util.KerberosUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.security.auth.login.AppConfigurationEntry; +import javax.security.auth.login.Configuration; +import java.io.File; +import java.util.HashMap; +import java.util.Map; + +/** + * + * JAAS configuration provider object that provides default LoginModule for various connectors that supports + * JAAS/SASL based Kerberos authentication. The implementation is inspired from Hadoop UGI class. + * + * Different connectors uses different login module name to implement JAAS based authentication support. + * For example, Kafka expects the login module name to be "kafkaClient" whereas ZooKeeper expect the + * name to be "client". This sets responsibility on the Flink cluster administrator to configure/provide right + * JAAS config entries. To simplify this requirement, we have introduced this abstraction that provides + * a standard lookup to get the login module entry for the JAAS based authentication to work. + * + * HDFS connector will not be impacted with this configuration since it uses UGI based mechanism to authenticate. + * + * Configuration + * + */ + +@Internal +public class JaasConfiguration extends Configuration { + + private static final Logger LOG = LoggerFactory.getLogger(JaasConfiguration.class); + + public static final String JAVA_VENDOR_NAME = System.getProperty("java.vendor"); + + public static final boolean IBM_JAVA; + + private static final Map debugOptions = new HashMap<>(); + + private static final Map kerberosCacheOptions = new HashMap<>(); + + private static final Map keytabKerberosOptions = new HashMap<>(); + + private static final AppConfigurationEntry userKerberosAce; + + private AppConfigurationEntry keytabKerberosAce = null; + + static { + + IBM_JAVA = JAVA_VENDOR_NAME.contains("IBM"); + + if(LOG.isDebugEnabled()) { + debugOptions.put("debug", "true"); + } + + if(IBM_JAVA) { + kerberosCacheOptions.put("useDefaultCcache", "true"); + } else { + kerberosCacheOptions.put("doNotPrompt", "true"); + kerberosCacheOptions.put("useTicketCache", "true"); + } + + String ticketCache = System.getenv("KRB5CCNAME"); + if(ticketCache != null) { + if(IBM_JAVA) { + System.setProperty("KRB5CCNAME", ticketCache); + } else { + kerberosCacheOptions.put("ticketCache", ticketCache); + } + } + + kerberosCacheOptions.put("renewTGT", "true"); + kerberosCacheOptions.putAll(debugOptions); + + userKerberosAce = new AppConfigurationEntry( + KerberosUtil.getKrb5LoginModuleName(), + AppConfigurationEntry.LoginModuleControlFlag.OPTIONAL, + kerberosCacheOptions); + + } + + protected JaasConfiguration(String keytab, String principal) { + + LOG.info("Initializing JAAS configuration instance. Parameters: {}, {}", keytab, principal); + + if(StringUtils.isBlank(keytab) && !StringUtils.isBlank(principal) || + (!StringUtils.isBlank(keytab) && StringUtils.isBlank(principal))){ + throw new RuntimeException("Both keytab and principal are required and cannot be empty"); + } + + if(!StringUtils.isBlank(keytab) && !StringUtils.isBlank(principal)) { + + if(IBM_JAVA) { + keytabKerberosOptions.put("useKeytab", prependFileUri(keytab)); + keytabKerberosOptions.put("credsType", "both"); + } else { + keytabKerberosOptions.put("keyTab", keytab); + keytabKerberosOptions.put("doNotPrompt", "true"); + keytabKerberosOptions.put("useKeyTab", "true"); + keytabKerberosOptions.put("storeKey", "true"); + } + + keytabKerberosOptions.put("principal", principal); + keytabKerberosOptions.put("refreshKrb5Config", "true"); + keytabKerberosOptions.putAll(debugOptions); + + keytabKerberosAce = new AppConfigurationEntry( + KerberosUtil.getKrb5LoginModuleName(), + AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, + keytabKerberosOptions); + } + } + + public static Map getKeytabKerberosOptions() { + return keytabKerberosOptions; + } + + private static String prependFileUri(String keytabPath) { + File f = new File(keytabPath); + return f.toURI().toString(); + } + + @Override + public AppConfigurationEntry[] getAppConfigurationEntry(String applicationName) { + + LOG.debug("JAAS configuration requested for the application entry: {}", applicationName); + + AppConfigurationEntry[] appConfigurationEntry; + + if(keytabKerberosAce != null) { + appConfigurationEntry = new AppConfigurationEntry[] {keytabKerberosAce, userKerberosAce}; + } else { + appConfigurationEntry = new AppConfigurationEntry[] {userKerberosAce}; + } + + return appConfigurationEntry; + } + +} \ No newline at end of file diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityContext.java b/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityContext.java new file mode 100644 index 0000000000000..4b8b69bc5f816 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityContext.java @@ -0,0 +1,313 @@ +/* + * 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.runtime.security; + +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.security.token.TokenIdentifier; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.security.auth.Subject; +import java.io.File; +import java.lang.reflect.Method; +import java.security.PrivilegedExceptionAction; +import java.util.Collection; + +/* + * Process-wide security context object which initializes UGI with appropriate security credentials and also it + * creates in-memory JAAS configuration object which will serve appropriate ApplicationConfigurationEntry for the + * connector login module implementation that authenticates Kerberos identity using SASL/JAAS based mechanism. + */ +@Internal +public class SecurityContext { + + private static final Logger LOG = LoggerFactory.getLogger(SecurityContext.class); + + public static final String JAAS_CONF_FILENAME = "flink-jaas.conf"; + + private static final String JAVA_SECURITY_AUTH_LOGIN_CONFIG = "java.security.auth.login.config"; + + private static final String ZOOKEEPER_SASL_CLIENT = "zookeeper.sasl.client"; + + private static final String ZOOKEEPER_SASL_CLIENT_USERNAME = "zookeeper.sasl.client.username"; + + private static SecurityContext installedContext; + + public static SecurityContext getInstalled() { return installedContext; } + + private UserGroupInformation ugi; + + SecurityContext(UserGroupInformation ugi) { + if(ugi == null) { + throw new RuntimeException("UGI passed cannot be null"); + } + this.ugi = ugi; + } + + public T runSecured(final FlinkSecuredRunner runner) throws Exception { + return ugi.doAs(new PrivilegedExceptionAction() { + @Override + public T run() throws Exception { + return runner.run(); + } + }); + } + + public static void install(SecurityConfiguration config) throws Exception { + + // perform static initialization of UGI, JAAS + if(installedContext != null) { + LOG.warn("overriding previous security context"); + } + + // establish the JAAS config + JaasConfiguration jaasConfig = new JaasConfiguration(config.keytab, config.principal); + javax.security.auth.login.Configuration.setConfiguration(jaasConfig); + + populateSystemSecurityProperties(config.flinkConf); + + // establish the UGI login user + UserGroupInformation.setConfiguration(config.hadoopConf); + + UserGroupInformation loginUser; + + if(UserGroupInformation.isSecurityEnabled() && + config.keytab != null && !StringUtils.isBlank(config.principal)) { + String keytabPath = (new File(config.keytab)).getAbsolutePath(); + + UserGroupInformation.loginUserFromKeytab(config.principal, keytabPath); + + loginUser = UserGroupInformation.getLoginUser(); + + // supplement with any available tokens + String fileLocation = System.getenv(UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION); + if(fileLocation != null) { + /* + * Use reflection API since the API semantics are not available in Hadoop1 profile. Below APIs are + * used in the context of reading the stored tokens from UGI. + * Credentials cred = Credentials.readTokenStorageFile(new File(fileLocation), config.hadoopConf); + * loginUser.addCredentials(cred); + */ + try { + Method readTokenStorageFileMethod = Credentials.class.getMethod("readTokenStorageFile", + File.class, org.apache.hadoop.conf.Configuration.class); + Credentials cred = (Credentials) readTokenStorageFileMethod.invoke(null,new File(fileLocation), + config.hadoopConf); + Method addCredentialsMethod = UserGroupInformation.class.getMethod("addCredentials", + Credentials.class); + addCredentialsMethod.invoke(loginUser,cred); + } catch(NoSuchMethodException e) { + LOG.warn("Could not find method implementations in the shaded jar. Exception: {}", e); + } + } + } else { + // login with current user credentials (e.g. ticket cache) + try { + //Use reflection API to get the login user object + //UserGroupInformation.loginUserFromSubject(null); + Method loginUserFromSubjectMethod = UserGroupInformation.class.getMethod("loginUserFromSubject", Subject.class); + Subject subject = null; + loginUserFromSubjectMethod.invoke(null,subject); + } catch(NoSuchMethodException e) { + LOG.warn("Could not find method implementations in the shaded jar. Exception: {}", e); + } + + loginUser = UserGroupInformation.getLoginUser(); + // note that the stored tokens are read automatically + } + + boolean delegationToken = false; + final Text HDFS_DELEGATION_KIND = new Text("HDFS_DELEGATION_TOKEN"); + Collection> usrTok = loginUser.getTokens(); + for(Token token : usrTok) { + final Text id = new Text(token.getIdentifier()); + LOG.debug("Found user token " + id + " with " + token); + if(token.getKind().equals(HDFS_DELEGATION_KIND)) { + delegationToken = true; + } + } + + if(UserGroupInformation.isSecurityEnabled() && !loginUser.hasKerberosCredentials()) { + //throw an error in non-yarn deployment if kerberos cache is not available + if(!delegationToken) { + LOG.error("Hadoop Security is enabled but current login user does not have Kerberos Credentials"); + throw new RuntimeException("Hadoop Security is enabled but current login user does not have Kerberos Credentials"); + } + } + + installedContext = new SecurityContext(loginUser); + } + + /* + * This method configures some of the system properties that are require for ZK and Kafka SASL authentication + * See: https://github.com/apache/kafka/blob/0.9.0/clients/src/main/java/org/apache/kafka/common/security/kerberos/Login.java#L289 + * See: https://github.com/sgroschupf/zkclient/blob/master/src/main/java/org/I0Itec/zkclient/ZkClient.java#L900 + * In this method, setting java.security.auth.login.config configuration is configured only to support ZK and + * Kafka current code behavior. + */ + private static void populateSystemSecurityProperties(Configuration configuration) { + + //required to be empty for Kafka but we will override the property + //with pseudo JAAS configuration file if SASL auth is enabled for ZK + System.setProperty(JAVA_SECURITY_AUTH_LOGIN_CONFIG, ""); + + if(configuration == null) { + return; + } + + boolean disableSaslClient = configuration.getBoolean(ConfigConstants.ZOOKEEPER_SASL_DISABLE, + ConfigConstants.DEFAULT_ZOOKEEPER_SASL_DISABLE); + if(disableSaslClient) { + LOG.info("SASL client auth for ZK will be disabled"); + //SASL auth is disabled by default but will be enabled if specified in configuration + System.setProperty(ZOOKEEPER_SASL_CLIENT,"false"); + return; + } + + String baseDir = configuration.getString(ConfigConstants.FLINK_BASE_DIR_PATH_KEY, null); + if(baseDir == null) { + String message = "SASL auth is enabled for ZK but unable to locate pseudo Jaas config " + + "since " + ConfigConstants.FLINK_BASE_DIR_PATH_KEY + " is not provided"; + LOG.error(message); + throw new IllegalConfigurationException(message); + } + + File f = new File(baseDir); + if(!f.exists() || !f.isDirectory()) { + LOG.error("Invalid flink base directory {} configuration provided", baseDir); + throw new IllegalConfigurationException("Invalid flink base directory configuration provided"); + } + + File jaasConfigFile = new File(f, JAAS_CONF_FILENAME); + + if (!jaasConfigFile.exists() || !jaasConfigFile.isFile()) { + + //check if there is a conf directory + File confDir = new File(f, "conf"); + if(!confDir.exists() || !confDir.isDirectory()) { + LOG.error("Could not locate " + JAAS_CONF_FILENAME); + throw new IllegalConfigurationException("Could not locate " + JAAS_CONF_FILENAME); + } + + jaasConfigFile = new File(confDir, JAAS_CONF_FILENAME); + + if (!jaasConfigFile.exists() || !jaasConfigFile.isFile()) { + LOG.error("Could not locate " + JAAS_CONF_FILENAME); + throw new IllegalConfigurationException("Could not locate " + JAAS_CONF_FILENAME); + } + } + + LOG.info("Enabling {} property with pseudo JAAS config file: {}", + JAVA_SECURITY_AUTH_LOGIN_CONFIG, jaasConfigFile); + + //ZK client module lookup the configuration to handle SASL. + //https://github.com/sgroschupf/zkclient/blob/master/src/main/java/org/I0Itec/zkclient/ZkClient.java#L900 + System.setProperty(JAVA_SECURITY_AUTH_LOGIN_CONFIG, jaasConfigFile.getAbsolutePath()); + System.setProperty(ZOOKEEPER_SASL_CLIENT,"true"); + + String zkSaslServiceName = configuration.getString(ConfigConstants.ZOOKEEPER_SASL_SERVICE_NAME, null); + if(!StringUtils.isBlank(zkSaslServiceName)) { + LOG.info("ZK SASL service name: {} is provided in the configuration", zkSaslServiceName); + System.setProperty(ZOOKEEPER_SASL_CLIENT_USERNAME,zkSaslServiceName); + } + + } + + /** + * Inputs for establishing the security context. + */ + public static class SecurityConfiguration { + + Configuration flinkConf; + + org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration(); + + String keytab; + + String principal; + + public String getKeytab() { + return keytab; + } + + public String getPrincipal() { + return principal; + } + + public SecurityConfiguration setFlinkConfiguration(Configuration flinkConf) { + + this.flinkConf = flinkConf; + + String keytab = flinkConf.getString(ConfigConstants.SECURITY_KEYTAB_KEY, null); + + String principal = flinkConf.getString(ConfigConstants.SECURITY_PRINCIPAL_KEY, null); + + validate(keytab, principal); + + LOG.debug("keytab {} and principal {} .", keytab, principal); + + this.keytab = keytab; + + this.principal = principal; + + return this; + } + + public SecurityConfiguration setHadoopConfiguration(org.apache.hadoop.conf.Configuration conf) { + this.hadoopConf = conf; + return this; + } + + private void validate(String keytab, String principal) { + + if(StringUtils.isBlank(keytab) && !StringUtils.isBlank(principal) || + !StringUtils.isBlank(keytab) && StringUtils.isBlank(principal)) { + if(StringUtils.isBlank(keytab)) { + LOG.warn("Keytab is null or empty"); + } + if(StringUtils.isBlank(principal)) { + LOG.warn("Principal is null or empty"); + } + throw new RuntimeException("Requires both keytab and principal to be provided"); + } + + if(!StringUtils.isBlank(keytab)) { + File keytabFile = new File(keytab); + if(!keytabFile.exists() || !keytabFile.isFile()) { + LOG.warn("Not a valid keytab: {} file", keytab); + throw new RuntimeException("Invalid keytab file: " + keytab + " passed"); + } + } + + } + } + + public interface FlinkSecuredRunner { + T run() throws Exception; + } + +} \ No newline at end of file diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index 9c844baa0add8..639c158b56be4 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -72,8 +72,8 @@ import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup import org.apache.flink.runtime.process.ProcessReaper import org.apache.flink.runtime.query.{KvStateMessage, UnknownKvStateLocation} import org.apache.flink.runtime.query.KvStateMessage.{LookupKvStateLocation, NotifyKvStateRegistered, NotifyKvStateUnregistered} -import org.apache.flink.runtime.security.SecurityUtils -import org.apache.flink.runtime.security.SecurityUtils.FlinkSecuredRunner +import org.apache.flink.runtime.security.SecurityContext.{FlinkSecuredRunner, SecurityConfiguration} +import org.apache.flink.runtime.security.{SecurityContext} import org.apache.flink.runtime.taskmanager.TaskManager import org.apache.flink.runtime.util._ import org.apache.flink.runtime.webmonitor.{WebMonitor, WebMonitorUtils} @@ -2062,26 +2062,18 @@ object JobManager { } // run the job manager + SecurityContext.install(new SecurityConfiguration().setFlinkConfiguration(configuration)) + try { - if (SecurityUtils.isSecurityEnabled) { - LOG.info("Security is enabled. Starting secure JobManager.") - SecurityUtils.runSecured(new FlinkSecuredRunner[Unit] { - override def run(): Unit = { - runJobManager( - configuration, - executionMode, - listeningHost, - listeningPortRange) - } - }) - } else { - LOG.info("Security is not enabled. Starting non-authenticated JobManager.") - runJobManager( - configuration, - executionMode, - listeningHost, - listeningPortRange) - } + SecurityContext.getInstalled.runSecured(new FlinkSecuredRunner[Unit] { + override def run(): Unit = { + runJobManager( + configuration, + executionMode, + listeningHost, + listeningPortRange) + } + }) } catch { case t: Throwable => LOG.error("Failed to run JobManager.", t) diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index 63a64a0f9f483..8534ee1b71565 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -71,8 +71,8 @@ import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup import org.apache.flink.runtime.process.ProcessReaper import org.apache.flink.runtime.query.KvStateRegistry import org.apache.flink.runtime.query.netty.{DisabledKvStateRequestStats, KvStateServer} -import org.apache.flink.runtime.security.SecurityUtils -import org.apache.flink.runtime.security.SecurityUtils.FlinkSecuredRunner +import org.apache.flink.runtime.security.SecurityContext.{FlinkSecuredRunner, SecurityConfiguration} +import org.apache.flink.runtime.security.SecurityContext import org.apache.flink.runtime.util._ import org.apache.flink.runtime.{FlinkActor, LeaderSessionMessageFilter, LogMessages} import org.apache.flink.util.{MathUtils, NetUtils} @@ -1521,19 +1521,14 @@ object TaskManager { val resourceId = ResourceID.generate() // run the TaskManager (if requested in an authentication enabled context) + SecurityContext.install(new SecurityConfiguration().setFlinkConfiguration(configuration)) + try { - if (SecurityUtils.isSecurityEnabled) { - LOG.info("Security is enabled. Starting secure TaskManager.") - SecurityUtils.runSecured(new FlinkSecuredRunner[Unit] { - override def run(): Unit = { - selectNetworkInterfaceAndRunTaskManager(configuration, resourceId, classOf[TaskManager]) - } - }) - } - else { - LOG.info("Security is not enabled. Starting non-authenticated TaskManager.") - selectNetworkInterfaceAndRunTaskManager(configuration, resourceId, classOf[TaskManager]) - } + SecurityContext.getInstalled.runSecured(new FlinkSecuredRunner[Unit] { + override def run(): Unit = { + selectNetworkInterfaceAndRunTaskManager(configuration, resourceId, classOf[TaskManager]) + } + }) } catch { case t: Throwable => @@ -1588,6 +1583,8 @@ object TaskManager { } } + conf.setString(ConfigConstants.FLINK_BASE_DIR_PATH_KEY, cliConfig.getConfigDir() + "/..") + conf } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/security/JaasConfigurationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/security/JaasConfigurationTest.java new file mode 100644 index 0000000000000..89e5ef9037e78 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/security/JaasConfigurationTest.java @@ -0,0 +1,52 @@ +/* + * 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.runtime.security; + +import org.apache.hadoop.security.authentication.util.KerberosUtil; +import org.junit.Test; + +import javax.security.auth.login.AppConfigurationEntry; + +import static org.junit.Assert.assertEquals; + +/** + * Tests for the {@link JaasConfiguration}. + */ +public class JaasConfigurationTest { + + @Test + public void testInvalidKerberosParams() { + String keytab = "user.keytab"; + String principal = null; + try { + new JaasConfiguration(keytab, principal); + } catch(RuntimeException re) { + assertEquals("Both keytab and principal are required and cannot be empty",re.getMessage()); + } + } + + @Test + public void testDefaultAceEntry() { + JaasConfiguration conf = new JaasConfiguration(null,null); + javax.security.auth.login.Configuration.setConfiguration(conf); + final AppConfigurationEntry[] entry = conf.getAppConfigurationEntry("test"); + AppConfigurationEntry ace = entry[0]; + assertEquals(ace.getLoginModuleName(), KerberosUtil.getKrb5LoginModuleName()); + } +} \ No newline at end of file diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/security/SecurityContextTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/security/SecurityContextTest.java new file mode 100644 index 0000000000000..5f3d76a71bef9 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/security/SecurityContextTest.java @@ -0,0 +1,77 @@ +/* + * 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.runtime.security; + +import org.apache.hadoop.security.UserGroupInformation; +import org.junit.Test; + +import java.lang.reflect.Method; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +/** + * Tests for the {@link SecurityContext}. + */ +public class SecurityContextTest { + + @Test + public void testCreateInsecureHadoopCtx() { + SecurityContext.SecurityConfiguration sc = new SecurityContext.SecurityConfiguration(); + try { + SecurityContext.install(sc); + assertEquals(UserGroupInformation.getLoginUser().getUserName(),getOSUserName()); + } catch (Exception e) { + fail(e.getMessage()); + } + } + + @Test + public void testInvalidUGIContext() { + try { + new SecurityContext(null); + } catch (RuntimeException re) { + assertEquals("UGI passed cannot be null",re.getMessage()); + } + } + + + private String getOSUserName() throws Exception { + String userName = ""; + String osName = System.getProperty( "os.name" ).toLowerCase(); + String className = null; + + if( osName.contains( "windows" ) ){ + className = "com.sun.security.auth.module.NTSystem"; + } + else if( osName.contains( "linux" ) ){ + className = "com.sun.security.auth.module.UnixSystem"; + } + else if( osName.contains( "solaris" ) || osName.contains( "sunos" ) ){ + className = "com.sun.security.auth.module.SolarisSystem"; + } + + if( className != null ){ + Class c = Class.forName( className ); + Method method = c.getDeclaredMethod( "getUsername" ); + Object o = c.newInstance(); + userName = (String) method.invoke( o ); + } + return userName; + } +} diff --git a/flink-streaming-connectors/flink-connector-filesystem/pom.xml b/flink-streaming-connectors/flink-connector-filesystem/pom.xml index 5712856b2a3dc..edf299d663a65 100644 --- a/flink-streaming-connectors/flink-connector-filesystem/pom.xml +++ b/flink-streaming-connectors/flink-connector-filesystem/pom.xml @@ -121,6 +121,42 @@ under the License. ${hadoop.version} + + org.apache.hadoop + hadoop-minikdc + ${minikdc.version} + + + + + + + + org.apache.felix + maven-bundle-plugin + 3.0.1 + true + true + + + + org.apache.maven.plugins + maven-surefire-plugin + + + 1 + false + + + + + diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkITCase.java b/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkITCase.java index 7ee75c1557302..c3c8df55183e1 100644 --- a/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkITCase.java +++ b/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkITCase.java @@ -46,6 +46,7 @@ import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.SequenceFile; @@ -57,6 +58,8 @@ import org.junit.ClassRule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.BufferedReader; import java.io.File; @@ -79,19 +82,24 @@ @Deprecated public class RollingSinkITCase extends StreamingMultipleProgramsTestBase { + protected static final Logger LOG = LoggerFactory.getLogger(RollingSinkITCase.class); + @ClassRule public static TemporaryFolder tempFolder = new TemporaryFolder(); - private static MiniDFSCluster hdfsCluster; - private static org.apache.hadoop.fs.FileSystem dfs; - private static String hdfsURI; + protected static MiniDFSCluster hdfsCluster; + protected static org.apache.hadoop.fs.FileSystem dfs; + protected static String hdfsURI; + protected static Configuration conf = new Configuration(); + protected static File dataDir; @BeforeClass public static void createHDFS() throws IOException { - Configuration conf = new Configuration(); - File dataDir = tempFolder.newFolder(); + LOG.info("In RollingSinkITCase: Starting MiniDFSCluster "); + + dataDir = tempFolder.newFolder(); conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, dataDir.getAbsolutePath()); MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); @@ -106,6 +114,7 @@ public static void createHDFS() throws IOException { @AfterClass public static void destroyHDFS() { + LOG.info("In RollingSinkITCase: tearing down MiniDFSCluster "); hdfsCluster.shutdown(); } diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java b/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java new file mode 100644 index 0000000000000..86cedaf307a7c --- /dev/null +++ b/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java @@ -0,0 +1,232 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.fs; + +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.security.SecurityContext; +import org.apache.flink.streaming.util.TestStreamEnvironment; +import org.apache.flink.test.util.SecureTestEnvironment; +import org.apache.flink.test.util.TestingSecurityContext; +import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.util.NetUtils; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.http.HttpConfig; +import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.security.UserGroupInformation; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Ignore; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_KEYTAB_FILE_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HTTP_POLICY_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY; + +/** + * Tests for running {@link RollingSinkSecuredITCase} which is an extension of {@link RollingSink} in secure environment + */ + +//The test is disabled since MiniDFS secure run requires lower order ports to be used. +//We can enable the test when the fix is available (HDFS-9213) +@Ignore +public class RollingSinkSecuredITCase extends RollingSinkITCase { + + protected static final Logger LOG = LoggerFactory.getLogger(RollingSinkSecuredITCase.class); + + /* + * override super class static methods to avoid creating MiniDFS and MiniFlink with wrong configurations + * and out-of-order sequence for secure cluster + */ + @BeforeClass + public static void setup() throws Exception {} + + @AfterClass + public static void teardown() throws Exception {} + + @BeforeClass + public static void createHDFS() throws IOException {} + + @AfterClass + public static void destroyHDFS() {} + + @BeforeClass + public static void startSecureCluster() throws Exception { + + LOG.info("starting secure cluster environment for testing"); + + dataDir = tempFolder.newFolder(); + + conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, dataDir.getAbsolutePath()); + + SecureTestEnvironment.prepare(tempFolder); + + populateSecureConfigurations(); + + Configuration flinkConfig = new Configuration(); + flinkConfig.setString(ConfigConstants.SECURITY_KEYTAB_KEY, + SecureTestEnvironment.getTestKeytab()); + flinkConfig.setString(ConfigConstants.SECURITY_PRINCIPAL_KEY, + SecureTestEnvironment.getHadoopServicePrincipal()); + + SecurityContext.SecurityConfiguration ctx = new SecurityContext.SecurityConfiguration(); + ctx.setFlinkConfiguration(flinkConfig); + ctx.setHadoopConfiguration(conf); + try { + TestingSecurityContext.install(ctx, SecureTestEnvironment.getClientSecurityConfigurationMap()); + } catch (Exception e) { + throw new RuntimeException("Exception occurred while setting up secure test context. Reason: {}", e); + } + + File hdfsSiteXML = new File(dataDir.getAbsolutePath() + "/hdfs-site.xml"); + + FileWriter writer = new FileWriter(hdfsSiteXML); + conf.writeXml(writer); + writer.flush(); + writer.close(); + + Map map = new HashMap(System.getenv()); + map.put("HADOOP_CONF_DIR", hdfsSiteXML.getParentFile().getAbsolutePath()); + TestBaseUtils.setEnv(map); + + + MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf); + builder.checkDataNodeAddrConfig(true); + builder.checkDataNodeHostConfig(true); + hdfsCluster = builder.build(); + + dfs = hdfsCluster.getFileSystem(); + + hdfsURI = "hdfs://" + + NetUtils.hostAndPortToUrlString(hdfsCluster.getURI().getHost(), hdfsCluster.getNameNodePort()) + + "/"; + + startSecureFlinkClusterWithRecoveryModeEnabled(); + } + + @AfterClass + public static void teardownSecureCluster() throws Exception { + LOG.info("tearing down secure cluster environment"); + + TestStreamEnvironment.unsetAsContext(); + stopCluster(cluster, TestBaseUtils.DEFAULT_TIMEOUT); + + hdfsCluster.shutdown(); + + SecureTestEnvironment.cleanup(); + } + + private static void populateSecureConfigurations() { + + String dataTransferProtection = "authentication"; + + SecurityUtil.setAuthenticationMethod(UserGroupInformation.AuthenticationMethod.KERBEROS, conf); + conf.set(DFS_NAMENODE_USER_NAME_KEY, SecureTestEnvironment.getHadoopServicePrincipal()); + conf.set(DFS_NAMENODE_KEYTAB_FILE_KEY, SecureTestEnvironment.getTestKeytab()); + conf.set(DFS_DATANODE_USER_NAME_KEY, SecureTestEnvironment.getHadoopServicePrincipal()); + conf.set(DFS_DATANODE_KEYTAB_FILE_KEY, SecureTestEnvironment.getTestKeytab()); + conf.set(DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY, SecureTestEnvironment.getHadoopServicePrincipal()); + + conf.setBoolean(DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true); + + conf.set("dfs.data.transfer.protection", dataTransferProtection); + + conf.set(DFS_HTTP_POLICY_KEY, HttpConfig.Policy.HTTP_ONLY.name()); + + conf.set(DFS_ENCRYPT_DATA_TRANSFER_KEY, "false"); + + conf.setInt("dfs.datanode.socket.write.timeout", 0); + + /* + * We ae setting the port number to privileged port - see HDFS-9213 + * This requires the user to have root privilege to bind to the port + * Use below command (ubuntu) to set privilege to java process for the + * bind() to work if the java process is not running as root. + * setcap 'cap_net_bind_service=+ep' /path/to/java + */ + conf.set(DFS_DATANODE_ADDRESS_KEY, "localhost:1002"); + conf.set(DFS_DATANODE_HOST_NAME_KEY, "localhost"); + conf.set(DFS_DATANODE_HTTP_ADDRESS_KEY, "localhost:1003"); + } + + private static void startSecureFlinkClusterWithRecoveryModeEnabled() { + try { + LOG.info("Starting Flink and ZK in secure mode"); + + dfs.mkdirs(new Path("/flink/checkpoints")); + dfs.mkdirs(new Path("/flink/recovery")); + + org.apache.flink.configuration.Configuration config = new org.apache.flink.configuration.Configuration(); + + config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1); + config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, DEFAULT_PARALLELISM); + config.setBoolean(ConfigConstants.LOCAL_START_WEBSERVER, false); + config.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, 3); + config.setString(ConfigConstants.RECOVERY_MODE, "zookeeper"); + config.setString(ConfigConstants.STATE_BACKEND, "filesystem"); + config.setString(ConfigConstants.ZOOKEEPER_CHECKPOINTS_PATH, hdfsURI + "/flink/checkpoints"); + config.setString(ConfigConstants.ZOOKEEPER_RECOVERY_PATH, hdfsURI + "/flink/recovery"); + config.setString("state.backend.fs.checkpointdir", hdfsURI + "/flink/checkpoints"); + + SecureTestEnvironment.populateFlinkSecureConfigurations(config); + + cluster = TestBaseUtils.startCluster(config, false); + TestStreamEnvironment.setAsContext(cluster, DEFAULT_PARALLELISM); + + } catch (Exception e) { + LOG.error("Exception occured while creating MiniFlink cluster. Reason: {}", e); + throw new RuntimeException(e); + } + } + + /* For secure cluster testing, it is enough to run only one test and override below test methods + * to keep the overall build time minimal + */ + @Override + public void testNonRollingSequenceFileWithoutCompressionWriter() throws Exception {} + + @Override + public void testNonRollingSequenceFileWithCompressionWriter() throws Exception {} + + @Override + public void testNonRollingAvroKeyValueWithoutCompressionWriter() throws Exception {} + + @Override + public void testNonRollingAvroKeyValueWithCompressionWriter() throws Exception {} + + @Override + public void testDateTimeRollingStringWriter() throws Exception {} + +} \ No newline at end of file diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/test/resources/log4j-test.properties b/flink-streaming-connectors/flink-connector-filesystem/src/test/resources/log4j-test.properties index fe60d94a0179f..5c22851a210fc 100644 --- a/flink-streaming-connectors/flink-connector-filesystem/src/test/resources/log4j-test.properties +++ b/flink-streaming-connectors/flink-connector-filesystem/src/test/resources/log4j-test.properties @@ -25,3 +25,5 @@ log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n # suppress the irrelevant (wrong) warnings from the netty channel handler log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger + +log4j.logger.org.apache.directory=OFF, testlogger \ No newline at end of file diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ProducerITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ProducerITCase.java index fc137198c3272..5c951db27e253 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ProducerITCase.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ProducerITCase.java @@ -21,7 +21,6 @@ import org.junit.Test; - @SuppressWarnings("serial") public class Kafka08ProducerITCase extends KafkaProducerTestBase { diff --git a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java index 864773a5f89b8..cbf3d06b7f199 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java @@ -80,6 +80,11 @@ public Properties getStandardProperties() { return standardProps; } + @Override + public Properties getSecureProperties() { + return null; + } + @Override public String getVersion() { return "0.8"; @@ -132,9 +137,14 @@ public int getBrokerId(KafkaServer server) { return server.socketServer().brokerId(); } + @Override + public boolean isSecureRunSupported() { + return false; + } + @Override - public void prepare(int numKafkaServers, Properties additionalServerProperties) { + public void prepare(int numKafkaServers, Properties additionalServerProperties, boolean secureMode) { this.additionalServerProperties = additionalServerProperties; File tempDir = new File(System.getProperty("java.io.tmpdir")); @@ -210,6 +220,7 @@ public void shutdown() { if (zookeeper != null) { try { zookeeper.stop(); + zookeeper.close(); } catch (Exception e) { LOG.warn("ZK.stop() failed", e); diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/pom.xml b/flink-streaming-connectors/flink-connector-kafka-0.9/pom.xml index 3b31de6c14c27..bfcde828fb875 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.9/pom.xml +++ b/flink-streaming-connectors/flink-connector-kafka-0.9/pom.xml @@ -134,6 +134,13 @@ under the License. test + + org.apache.hadoop + hadoop-minikdc + ${minikdc.version} + test + + @@ -180,6 +187,17 @@ under the License. -Xms256m -Xmx1000m -Dlog4j.configuration=${log4j.configuration} -Dmvn.forkNumber=${surefire.forkNumber} -XX:-UseGCOverheadLimit + + + org.apache.felix + maven-bundle-plugin + 3.0.1 + true + true + diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java index 957833de55974..16ddcdc389b9f 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ITCase.java @@ -21,12 +21,12 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.junit.Test; +import java.util.Properties; import java.util.UUID; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; - public class Kafka09ITCase extends KafkaConsumerTestBase { // ------------------------------------------------------------------------ @@ -131,11 +131,15 @@ public void testMetrics() throws Throwable { public void testJsonTableSource() throws Exception { String topic = UUID.randomUUID().toString(); + Properties props = new Properties(); + props.putAll(standardProps); + props.putAll(secureProps); + // Names and types are determined in the actual test method of the // base test class. Kafka09JsonTableSource tableSource = new Kafka09JsonTableSource( topic, - standardProps, + props, new String[] { "long", "string", @@ -159,11 +163,15 @@ public void testJsonTableSource() throws Exception { public void testJsonTableSourceWithFailOnMissingField() throws Exception { String topic = UUID.randomUUID().toString(); + Properties props = new Properties(); + props.putAll(standardProps); + props.putAll(secureProps); + // Names and types are determined in the actual test method of the // base test class. Kafka09JsonTableSource tableSource = new Kafka09JsonTableSource( topic, - standardProps, + props, new String[] { "long", "string", diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ProducerITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ProducerITCase.java index 1288347e28ac5..ae4f5b247f0db 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ProducerITCase.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ProducerITCase.java @@ -21,7 +21,6 @@ import org.junit.Test; - @SuppressWarnings("serial") public class Kafka09ProducerITCase extends KafkaProducerTestBase { diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09SecureRunITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09SecureRunITCase.java new file mode 100644 index 0000000000000..d12ec65af4cbc --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09SecureRunITCase.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.kafka; + +import org.apache.flink.test.util.SecureTestEnvironment; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +/* + * Kafka Secure Connection (kerberos) IT test case + */ +public class Kafka09SecureRunITCase extends KafkaConsumerTestBase { + + protected static final Logger LOG = LoggerFactory.getLogger(Kafka09SecureRunITCase.class); + + @BeforeClass + public static void prepare() throws IOException, ClassNotFoundException { + LOG.info("-------------------------------------------------------------------------"); + LOG.info(" Starting Kafka09SecureRunITCase "); + LOG.info("-------------------------------------------------------------------------"); + + SecureTestEnvironment.prepare(tempFolder); + SecureTestEnvironment.populateFlinkSecureConfigurations(getFlinkConfiguration()); + + startClusters(true); + } + + @AfterClass + public static void shutDownServices() { + shutdownClusters(); + SecureTestEnvironment.cleanup(); + } + + + //timeout interval is large since in Travis, ZK connection timeout occurs frequently + //The timeout for the test case is 2 times timeout of ZK connection + @Test(timeout = 600000) + public void testMultipleTopics() throws Exception { + runProduceConsumeMultipleTopics(); + } + +} \ No newline at end of file diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java index 0dbe865792efb..213ba4a31cb60 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java @@ -65,6 +65,9 @@ public class KafkaTestEnvironmentImpl extends KafkaTestEnvironment { private String brokerConnectionString = ""; private Properties standardProps; private Properties additionalServerProperties; + private boolean secureMode = false; + // 6 seconds is default. Seems to be too small for travis. 30 seconds + private String zkTimeout = "30000"; public String getBrokerConnectionString() { return brokerConnectionString; @@ -131,8 +134,22 @@ public int getBrokerId(KafkaServer server) { } @Override - public void prepare(int numKafkaServers, Properties additionalServerProperties) { + public boolean isSecureRunSupported() { + return true; + } + + @Override + public void prepare(int numKafkaServers, Properties additionalServerProperties, boolean secureMode) { + + //increase the timeout since in Travis ZK connection takes long time for secure connection. + if(secureMode) { + //run only one kafka server to avoid multiple ZK connections from many instances - Travis timeout + numKafkaServers = 1; + zkTimeout = String.valueOf(Integer.parseInt(zkTimeout) * 15); + } + this.additionalServerProperties = additionalServerProperties; + this.secureMode = secureMode; File tempDir = new File(System.getProperty("java.io.tmpdir")); tmpZkDir = new File(tempDir, "kafkaITcase-zk-dir-" + (UUID.randomUUID().toString())); @@ -155,6 +172,7 @@ public void prepare(int numKafkaServers, Properties additionalServerProperties) LOG.info("Starting Zookeeper"); zookeeper = new TestingServer(-1, tmpZkDir); zookeeperConnectionString = zookeeper.getConnectString(); + LOG.info("zookeeperConnectionString: {}", zookeeperConnectionString); LOG.info("Starting KafkaServer"); brokers = new ArrayList<>(numKafkaServers); @@ -163,7 +181,11 @@ public void prepare(int numKafkaServers, Properties additionalServerProperties) brokers.add(getKafkaServer(i, tmpKafkaDirs.get(i))); SocketServer socketServer = brokers.get(i).socketServer(); - brokerConnectionString += hostAndPortToUrlString(KafkaTestEnvironment.KAFKA_HOST, brokers.get(i).socketServer().boundPort(SecurityProtocol.PLAINTEXT)) + ","; + if(secureMode) { + brokerConnectionString += hostAndPortToUrlString(KafkaTestEnvironment.KAFKA_HOST, brokers.get(i).socketServer().boundPort(SecurityProtocol.SASL_PLAINTEXT)) + ","; + } else { + brokerConnectionString += hostAndPortToUrlString(KafkaTestEnvironment.KAFKA_HOST, brokers.get(i).socketServer().boundPort(SecurityProtocol.PLAINTEXT)) + ","; + } } LOG.info("ZK and KafkaServer started."); @@ -173,15 +195,18 @@ public void prepare(int numKafkaServers, Properties additionalServerProperties) fail("Test setup failed: " + t.getMessage()); } + LOG.info("brokerConnectionString --> {}", brokerConnectionString); + standardProps = new Properties(); standardProps.setProperty("zookeeper.connect", zookeeperConnectionString); standardProps.setProperty("bootstrap.servers", brokerConnectionString); standardProps.setProperty("group.id", "flink-tests"); standardProps.setProperty("auto.commit.enable", "false"); - standardProps.setProperty("zookeeper.session.timeout.ms", "30000"); // 6 seconds is default. Seems to be too small for travis. - standardProps.setProperty("zookeeper.connection.timeout.ms", "30000"); + standardProps.setProperty("zookeeper.session.timeout.ms", zkTimeout); + standardProps.setProperty("zookeeper.connection.timeout.ms", zkTimeout); standardProps.setProperty("auto.offset.reset", "earliest"); // read from the beginning. (earliest is kafka 0.9 value) standardProps.setProperty("fetch.message.max.bytes", "256"); // make a lot of fetches (MESSAGES MUST BE SMALLER!) + } @Override @@ -196,6 +221,7 @@ public void shutdown() { if (zookeeper != null) { try { zookeeper.stop(); + zookeeper.close(); } catch (Exception e) { LOG.warn("ZK.stop() failed", e); @@ -224,6 +250,7 @@ public void shutdown() { } public ZkUtils getZkUtils() { + LOG.info("In getZKUtils:: zookeeperConnectionString = {}", zookeeperConnectionString); ZkClient creator = new ZkClient(zookeeperConnectionString, Integer.valueOf(standardProps.getProperty("zookeeper.session.timeout.ms")), Integer.valueOf(standardProps.getProperty("zookeeper.connection.timeout.ms")), new ZooKeeperStringSerializer()); return ZkUtils.apply(creator, false); @@ -241,23 +268,37 @@ public void createTestTopic(String topic, int numberOfPartitions, int replicatio zkUtils.close(); } + LOG.info("Topic {} create request is successfully posted", topic); + // validate that the topic has been created - final long deadline = System.currentTimeMillis() + 30000; + final long deadline = System.currentTimeMillis() + Integer.parseInt(zkTimeout); do { try { - Thread.sleep(100); + if(secureMode) { + //increase wait time since in Travis ZK timeout occurs frequently + int wait = Integer.parseInt(zkTimeout) / 100; + LOG.info("waiting for {} msecs before the topic {} can be checked", wait, topic); + Thread.sleep(wait); + } else { + Thread.sleep(100); + } + } catch (InterruptedException e) { // restore interrupted state } // we could use AdminUtils.topicExists(zkUtils, topic) here, but it's results are // not always correct. + LOG.info("Validating if the topic {} has been created or not", topic); + // create a new ZK utils connection ZkUtils checkZKConn = getZkUtils(); if(AdminUtils.topicExists(checkZKConn, topic)) { + LOG.info("topic {} has been created successfully", topic); checkZKConn.close(); return; } + LOG.info("topic {} has not been created yet. Will check again...", topic); checkZKConn.close(); } while (System.currentTimeMillis() < deadline); @@ -296,8 +337,8 @@ protected KafkaServer getKafkaServer(int brokerId, File tmpFolder) throws Except kafkaProperties.put("replica.fetch.max.bytes", String.valueOf(50 * 1024 * 1024)); // for CI stability, increase zookeeper session timeout - kafkaProperties.put("zookeeper.session.timeout.ms", "30000"); - kafkaProperties.put("zookeeper.connection.timeout.ms", "30000"); + kafkaProperties.put("zookeeper.session.timeout.ms", zkTimeout); + kafkaProperties.put("zookeeper.connection.timeout.ms", zkTimeout); if(additionalServerProperties != null) { kafkaProperties.putAll(additionalServerProperties); } @@ -307,6 +348,15 @@ protected KafkaServer getKafkaServer(int brokerId, File tmpFolder) throws Except for (int i = 1; i <= numTries; i++) { int kafkaPort = NetUtils.getAvailablePort(); kafkaProperties.put("port", Integer.toString(kafkaPort)); + + //to support secure kafka cluster + if(secureMode) { + LOG.info("Adding Kafka secure configurations"); + kafkaProperties.put("listeners", "SASL_PLAINTEXT://" + KAFKA_HOST + ":" + kafkaPort); + kafkaProperties.put("advertised.listeners", "SASL_PLAINTEXT://" + KAFKA_HOST + ":" + kafkaPort); + kafkaProperties.putAll(getSecureProperties()); + } + KafkaConfig kafkaConfig = new KafkaConfig(kafkaProperties); try { @@ -329,4 +379,19 @@ protected KafkaServer getKafkaServer(int brokerId, File tmpFolder) throws Except throw new Exception("Could not start Kafka after " + numTries + " retries due to port conflicts."); } + public Properties getSecureProperties() { + Properties prop = new Properties(); + if(secureMode) { + prop.put("security.inter.broker.protocol", "SASL_PLAINTEXT"); + prop.put("security.protocol", "SASL_PLAINTEXT"); + prop.put("sasl.kerberos.service.name", "kafka"); + + //add special timeout for Travis + prop.setProperty("zookeeper.session.timeout.ms", zkTimeout); + prop.setProperty("zookeeper.connection.timeout.ms", zkTimeout); + prop.setProperty("metadata.fetch.timeout.ms","120000"); + } + return prop; + } + } diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/resources/log4j-test.properties b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/resources/log4j-test.properties index fbeb110350f5f..4ac1773190f18 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/resources/log4j-test.properties +++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/resources/log4j-test.properties @@ -28,3 +28,5 @@ log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger log4j.logger.org.apache.zookeeper=OFF, testlogger log4j.logger.state.change.logger=OFF, testlogger log4j.logger.kafka=OFF, testlogger + +log4j.logger.org.apache.directory=OFF, testlogger \ No newline at end of file diff --git a/flink-streaming-connectors/flink-connector-kafka-base/pom.xml b/flink-streaming-connectors/flink-connector-kafka-base/pom.xml index 49d630f03e87c..ef71bde9d74c4 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/pom.xml +++ b/flink-streaming-connectors/flink-connector-kafka-base/pom.xml @@ -161,6 +161,14 @@ under the License. test-jar test + + + org.apache.hadoop + hadoop-minikdc + ${minikdc.version} + test + + @@ -187,6 +195,17 @@ under the License. + + + org.apache.felix + maven-bundle-plugin + 3.0.1 + true + true + diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java index 920f15bd4ffc8..a87ff8ab1c8a3 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java @@ -181,6 +181,7 @@ public void runFailOnNoBrokerTest() throws Exception { properties.setProperty("session.timeout.ms", "2000"); properties.setProperty("fetch.max.wait.ms", "2000"); properties.setProperty("heartbeat.interval.ms", "1000"); + properties.putAll(secureProps); FlinkKafkaConsumerBase source = kafkaServer.getConsumer("doesntexist", new SimpleStringSchema(), properties); DataStream stream = see.addSource(source); stream.print(); @@ -275,6 +276,7 @@ public void cancel() { }); Properties producerProperties = FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings); producerProperties.setProperty("retries", "3"); + producerProperties.putAll(secureProps); FlinkKafkaProducerBase> prod = kafkaServer.getProducer(topic, new KeyedSerializationSchemaWrapper<>(sinkSchema), producerProperties, null); stream.addSink(prod); @@ -283,7 +285,11 @@ public void cancel() { List topics = new ArrayList<>(); topics.add(topic); topics.add(additionalEmptyTopic); - FlinkKafkaConsumerBase> source = kafkaServer.getConsumer(topics, sourceSchema, standardProps); + + Properties props = new Properties(); + props.putAll(standardProps); + props.putAll(secureProps); + FlinkKafkaConsumerBase> source = kafkaServer.getConsumer(topics, sourceSchema, props); DataStreamSource> consuming = env.addSource(source).setParallelism(parallelism); @@ -371,7 +377,11 @@ public void runOneToOneExactlyOnceTest() throws Exception { env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); env.getConfig().disableSysoutLogging(); - FlinkKafkaConsumerBase kafkaSource = kafkaServer.getConsumer(topic, schema, standardProps); + Properties props = new Properties(); + props.putAll(standardProps); + props.putAll(secureProps); + + FlinkKafkaConsumerBase kafkaSource = kafkaServer.getConsumer(topic, schema, props); env .addSource(kafkaSource) @@ -416,7 +426,10 @@ public void runOneSourceMultiplePartitionsExactlyOnceTest() throws Exception { env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); env.getConfig().disableSysoutLogging(); - FlinkKafkaConsumerBase kafkaSource = kafkaServer.getConsumer(topic, schema, standardProps); + Properties props = new Properties(); + props.putAll(standardProps); + props.putAll(secureProps); + FlinkKafkaConsumerBase kafkaSource = kafkaServer.getConsumer(topic, schema, props); env .addSource(kafkaSource) @@ -463,7 +476,10 @@ public void runMultipleSourcesOnePartitionExactlyOnceTest() throws Exception { env.getConfig().disableSysoutLogging(); env.setBufferTimeout(0); - FlinkKafkaConsumerBase kafkaSource = kafkaServer.getConsumer(topic, schema, standardProps); + Properties props = new Properties(); + props.putAll(standardProps); + props.putAll(secureProps); + FlinkKafkaConsumerBase kafkaSource = kafkaServer.getConsumer(topic, schema, props); env .addSource(kafkaSource) @@ -506,7 +522,10 @@ public void run() { env.enableCheckpointing(100); env.getConfig().disableSysoutLogging(); - FlinkKafkaConsumerBase source = kafkaServer.getConsumer(topic, new SimpleStringSchema(), standardProps); + Properties props = new Properties(); + props.putAll(standardProps); + props.putAll(secureProps); + FlinkKafkaConsumerBase source = kafkaServer.getConsumer(topic, new SimpleStringSchema(), props); env.addSource(source).addSink(new DiscardingSink()); @@ -577,7 +596,10 @@ public void run() { env.enableCheckpointing(100); env.getConfig().disableSysoutLogging(); - FlinkKafkaConsumerBase source = kafkaServer.getConsumer(topic, new SimpleStringSchema(), standardProps); + Properties props = new Properties(); + props.putAll(standardProps); + props.putAll(secureProps); + FlinkKafkaConsumerBase source = kafkaServer.getConsumer(topic, new SimpleStringSchema(), props); env.addSource(source).addSink(new DiscardingSink()); @@ -629,7 +651,10 @@ public void runFailOnDeployTest() throws Exception { env.setParallelism(12); // needs to be more that the mini cluster has slots env.getConfig().disableSysoutLogging(); - FlinkKafkaConsumerBase kafkaSource = kafkaServer.getConsumer(topic, schema, standardProps); + Properties props = new Properties(); + props.putAll(standardProps); + props.putAll(secureProps); + FlinkKafkaConsumerBase kafkaSource = kafkaServer.getConsumer(topic, schema, props); env .addSource(kafkaSource) @@ -700,15 +725,19 @@ public void cancel() { Tuple2WithTopicSchema schema = new Tuple2WithTopicSchema(env.getConfig()); - stream.addSink(kafkaServer.getProducer("dummy", schema, standardProps, null)); + Properties props = new Properties(); + props.putAll(standardProps); + props.putAll(secureProps); + + stream.addSink(kafkaServer.getProducer("dummy", schema, props, null)); env.execute("Write to topics"); // run second job consuming from multiple topics env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort); env.getConfig().disableSysoutLogging(); - - stream = env.addSource(kafkaServer.getConsumer(topics, schema, standardProps)); + + stream = env.addSource(kafkaServer.getConsumer(topics, schema, props)); stream.flatMap(new FlatMapFunction, Integer>() { Map countPerTopic = new HashMap<>(NUM_TOPICS); @@ -787,6 +816,10 @@ public void runJsonTableSource(String topic, KafkaTableSource kafkaTableSource) // Produce serialized JSON data createTestTopic(topic, 1, 1); + Properties props = new Properties(); + props.putAll(standardProps); + props.putAll(secureProps); + StreamExecutionEnvironment env = StreamExecutionEnvironment .createRemoteEnvironment("localhost", flinkPort); env.getConfig().disableSysoutLogging(); @@ -805,7 +838,7 @@ public void cancel() { }).addSink(kafkaServer.getProducer( topic, new ByteArraySerializationSchema(), - standardProps, + props, null)); // Execute blocks @@ -940,6 +973,7 @@ public void runBigRecordTestTopology() throws Exception { consumerProps.setProperty("fetch.message.max.bytes", Integer.toString(1024 * 1024 * 14)); consumerProps.setProperty("max.partition.fetch.bytes", Integer.toString(1024 * 1024 * 14)); // for the new fetcher consumerProps.setProperty("queued.max.message.chunks", "1"); + consumerProps.putAll(secureProps); FlinkKafkaConsumerBase> source = kafkaServer.getConsumer(topic, serSchema, consumerProps); DataStreamSource> consuming = env.addSource(source); @@ -969,6 +1003,7 @@ public void invoke(Tuple2 value) throws Exception { Properties producerProps = new Properties(); producerProps.setProperty("max.request.size", Integer.toString(1024 * 1024 * 15)); producerProps.setProperty("retries", "3"); + producerProps.putAll(secureProps); producerProps.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerConnectionStrings); DataStream> stream = env.addSource(new RichSourceFunction>() { @@ -1047,8 +1082,10 @@ public void runBrokerFailureTest() throws Exception { env.setRestartStrategy(RestartStrategies.noRestart()); env.getConfig().disableSysoutLogging(); - - FlinkKafkaConsumerBase kafkaSource = kafkaServer.getConsumer(topic, schema, standardProps); + Properties props = new Properties(); + props.putAll(standardProps); + props.putAll(secureProps); + FlinkKafkaConsumerBase kafkaSource = kafkaServer.getConsumer(topic, schema, props); env .addSource(kafkaSource) @@ -1097,6 +1134,7 @@ public void cancel() { KeyedSerializationSchema> schema = new TypeInformationKeyValueSerializationSchema<>(Long.class, PojoValue.class, env.getConfig()); Properties producerProperties = FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings); producerProperties.setProperty("retries", "3"); + producerProperties.putAll(secureProps); kvStream.addSink(kafkaServer.getProducer(topic, schema, producerProperties, null)); env.execute("Write KV to Kafka"); @@ -1110,7 +1148,10 @@ public void cancel() { KeyedDeserializationSchema> readSchema = new TypeInformationKeyValueSerializationSchema<>(Long.class, PojoValue.class, env.getConfig()); - DataStream> fromKafka = env.addSource(kafkaServer.getConsumer(topic, readSchema, standardProps)); + Properties props = new Properties(); + props.putAll(standardProps); + props.putAll(secureProps); + DataStream> fromKafka = env.addSource(kafkaServer.getConsumer(topic, readSchema, props)); fromKafka.flatMap(new RichFlatMapFunction, Object>() { long counter = 0; @Override @@ -1178,6 +1219,8 @@ public void cancel() { Properties producerProperties = FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings); producerProperties.setProperty("retries", "3"); + producerProperties.putAll(secureProps); + kvStream.addSink(kafkaServer.getProducer(topic, schema, producerProperties, null)); env.execute("Write deletes to Kafka"); @@ -1189,7 +1232,10 @@ public void cancel() { env.getConfig().setRestartStrategy(RestartStrategies.noRestart()); env.getConfig().disableSysoutLogging(); - DataStream> fromKafka = env.addSource(kafkaServer.getConsumer(topic, schema, standardProps)); + Properties props = new Properties(); + props.putAll(standardProps); + props.putAll(secureProps); + DataStream> fromKafka = env.addSource(kafkaServer.getConsumer(topic, schema, props)); fromKafka.flatMap(new RichFlatMapFunction, Object>() { long counter = 0; @@ -1226,7 +1272,11 @@ public void runEndOfStreamTest() throws Exception { env1.getConfig().setRestartStrategy(RestartStrategies.noRestart()); env1.getConfig().disableSysoutLogging(); - DataStream> fromKafka = env1.addSource(kafkaServer.getConsumer(topic, new FixedNumberDeserializationSchema(ELEMENT_COUNT), standardProps)); + Properties props = new Properties(); + props.putAll(standardProps); + props.putAll(secureProps); + + DataStream> fromKafka = env1.addSource(kafkaServer.getConsumer(topic, new FixedNumberDeserializationSchema(ELEMENT_COUNT), props)); fromKafka.flatMap(new FlatMapFunction, Void>() { @Override public void flatMap(Tuple2 value, Collector out) throws Exception { @@ -1262,8 +1312,12 @@ public void run() { env1.getConfig().disableSysoutLogging(); env1.disableOperatorChaining(); // let the source read everything into the network buffers + Properties props = new Properties(); + props.putAll(standardProps); + props.putAll(secureProps); + TypeInformationSerializationSchema> schema = new TypeInformationSerializationSchema<>(TypeInfoParser.>parse("Tuple2"), env1.getConfig()); - DataStream> fromKafka = env1.addSource(kafkaServer.getConsumer(topic, schema, standardProps)); + DataStream> fromKafka = env1.addSource(kafkaServer.getConsumer(topic, schema, props)); fromKafka.flatMap(new FlatMapFunction, Void>() { @Override public void flatMap(Tuple2 value, Collector out) throws Exception {// no op @@ -1288,7 +1342,7 @@ public void cancel() { } }); - fromGen.addSink(kafkaServer.getProducer(topic, new KeyedSerializationSchemaWrapper<>(schema), standardProps, null)); + fromGen.addSink(kafkaServer.getProducer(topic, new KeyedSerializationSchemaWrapper<>(schema), props, null)); env1.execute("Metrics test job"); } catch(Throwable t) { @@ -1403,6 +1457,7 @@ protected void readSequence(StreamExecutionEnvironment env, Properties cc, final TypeInformationSerializationSchema> deser = new TypeInformationSerializationSchema<>(intIntTupleType, env.getConfig()); + cc.putAll(secureProps); // create the consumer FlinkKafkaConsumerBase> consumer = kafkaServer.getConsumer(topicName, deser, cc); @@ -1505,6 +1560,7 @@ public void cancel() { // the producer must not produce duplicates Properties producerProperties = FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings); producerProperties.setProperty("retries", "0"); + producerProperties.putAll(secureProps); stream.addSink(kafkaServer.getProducer( topicName, serSchema, producerProperties, @@ -1537,7 +1593,7 @@ public void cancel() { Properties readProps = (Properties) standardProps.clone(); readProps.setProperty("group.id", "flink-tests-validator"); - + readProps.putAll(secureProps); FlinkKafkaConsumerBase> consumer = kafkaServer.getConsumer(topicName, deserSchema, readProps); readEnv @@ -1672,6 +1728,7 @@ private static void printTopic(String topicName, int elements,DeserializationSch newProps.setProperty("group.id", "topic-printer"+ UUID.randomUUID().toString()); newProps.setProperty("auto.offset.reset", "smallest"); newProps.setProperty("zookeeper.connect", standardProps.getProperty("zookeeper.connect")); + newProps.putAll(secureProps); ConsumerConfig printerConfig = new ConsumerConfig(newProps); printTopic(topicName, printerConfig, deserializer, elements); @@ -1893,8 +1950,11 @@ private DataStreamSink> runPunctuatedComsumer(StreamExecut TypeInformationSerializationSchema> sourceSchema = new TypeInformationSerializationSchema<>(inputTypeInfo, env.getConfig()); + Properties props = new Properties(); + props.putAll(standardProps); + props.putAll(secureProps); FlinkKafkaConsumerBase> source = kafkaServer - .getConsumer(topics, sourceSchema, standardProps) + .getConsumer(topics, sourceSchema, props) .assignTimestampsAndWatermarks(new TestPunctuatedTSExtractor()); DataStreamSource> consuming = env.setParallelism(1).addSource(source); diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java index 14e74f18e4a18..5bcf406533e83 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java @@ -34,6 +34,7 @@ import java.io.Serializable; +import java.util.Properties; import static org.apache.flink.test.util.TestUtils.tryExecute; import static org.junit.Assert.assertEquals; @@ -102,17 +103,24 @@ public void cancel() { } }) .setParallelism(1); + + Properties props = new Properties(); + props.putAll(FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings)); + props.putAll(secureProps); // sink partitions into stream.addSink(kafkaServer.getProducer(topic, new KeyedSerializationSchemaWrapper<>(serSchema), - FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings), + props, new CustomPartitioner(parallelism))) .setParallelism(parallelism); // ------ consuming topology --------- - - FlinkKafkaConsumerBase> source = kafkaServer.getConsumer(topic, deserSchema, standardProps); + + Properties consumerProps = new Properties(); + consumerProps.putAll(standardProps); + consumerProps.putAll(secureProps); + FlinkKafkaConsumerBase> source = kafkaServer.getConsumer(topic, deserSchema, consumerProps); env.addSource(source).setParallelism(parallelism) diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java index c4949ff83096c..9236e78bb1836 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java @@ -36,6 +36,8 @@ import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -60,29 +62,39 @@ public class KafkaShortRetentionTestBase implements Serializable { private static Properties standardProps; private static LocalFlinkMiniCluster flink; + @ClassRule + public static TemporaryFolder tempFolder = new TemporaryFolder(); + + protected static Properties secureProps = new Properties(); + @BeforeClass public static void prepare() throws IOException, ClassNotFoundException { LOG.info("-------------------------------------------------------------------------"); LOG.info(" Starting KafkaShortRetentionTestBase "); LOG.info("-------------------------------------------------------------------------"); + Configuration flinkConfig = new Configuration(); + // dynamically load the implementation for the test Class clazz = Class.forName("org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironmentImpl"); kafkaServer = (KafkaTestEnvironment) InstantiationUtil.instantiate(clazz); LOG.info("Starting KafkaTestBase.prepare() for Kafka " + kafkaServer.getVersion()); + if(kafkaServer.isSecureRunSupported()) { + secureProps = kafkaServer.getSecureProperties(); + } + Properties specificProperties = new Properties(); specificProperties.setProperty("log.retention.hours", "0"); specificProperties.setProperty("log.retention.minutes", "0"); specificProperties.setProperty("log.retention.ms", "250"); specificProperties.setProperty("log.retention.check.interval.ms", "100"); - kafkaServer.prepare(1, specificProperties); + kafkaServer.prepare(1, specificProperties, false); standardProps = kafkaServer.getStandardProperties(); // start also a re-usable Flink mini cluster - Configuration flinkConfig = new Configuration(); flinkConfig.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1); flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 8); flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 16); @@ -98,6 +110,8 @@ public static void shutDownServices() { flink.shutdown(); } kafkaServer.shutdown(); + + secureProps.clear(); } /** @@ -151,12 +165,17 @@ public void cancel() { running = false; } }); - stream.addSink(kafkaServer.getProducer(topic, new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), standardProps, null)); + + Properties props = new Properties(); + props.putAll(standardProps); + props.putAll(secureProps); + + stream.addSink(kafkaServer.getProducer(topic, new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), props, null)); // ----------- add consumer dataflow ---------- NonContinousOffsetsDeserializationSchema deserSchema = new NonContinousOffsetsDeserializationSchema(); - FlinkKafkaConsumerBase source = kafkaServer.getConsumer(topic, deserSchema, standardProps); + FlinkKafkaConsumerBase source = kafkaServer.getConsumer(topic, deserSchema, props); DataStreamSource consuming = env.addSource(source); consuming.addSink(new DiscardingSink()); @@ -224,6 +243,7 @@ public void runFailOnAutoOffsetResetNone() throws Exception { Properties customProps = new Properties(); customProps.putAll(standardProps); + customProps.putAll(secureProps); customProps.setProperty("auto.offset.reset", "none"); // test that "none" leads to an exception FlinkKafkaConsumerBase source = kafkaServer.getConsumer(topic, new SimpleStringSchema(), customProps); @@ -255,6 +275,7 @@ public void runFailOnAutoOffsetResetNoneEager() throws Exception { Properties customProps = new Properties(); customProps.putAll(standardProps); + customProps.putAll(secureProps); customProps.setProperty("auto.offset.reset", "none"); // test that "none" leads to an exception try { diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java index 771db17120702..afdd158e35eca 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java @@ -31,6 +31,8 @@ import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -73,61 +75,90 @@ public abstract class KafkaTestBase extends TestLogger { protected static KafkaTestEnvironment kafkaServer; + @ClassRule + public static TemporaryFolder tempFolder = new TemporaryFolder(); + + protected static Properties secureProps = new Properties(); + // ------------------------------------------------------------------------ // Setup and teardown of the mini clusters // ------------------------------------------------------------------------ @BeforeClass public static void prepare() throws IOException, ClassNotFoundException { + LOG.info("-------------------------------------------------------------------------"); LOG.info(" Starting KafkaTestBase "); LOG.info("-------------------------------------------------------------------------"); - + startClusters(false); - // dynamically load the implementation for the test - Class clazz = Class.forName("org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironmentImpl"); - kafkaServer = (KafkaTestEnvironment) InstantiationUtil.instantiate(clazz); + } - LOG.info("Starting KafkaTestBase.prepare() for Kafka " + kafkaServer.getVersion()); + @AfterClass + public static void shutDownServices() { - kafkaServer.prepare(NUMBER_OF_KAFKA_SERVERS); + LOG.info("-------------------------------------------------------------------------"); + LOG.info(" Shut down KafkaTestBase "); + LOG.info("-------------------------------------------------------------------------"); - standardProps = kafkaServer.getStandardProperties(); - brokerConnectionStrings = kafkaServer.getBrokerConnectionString(); + shutdownClusters(); - // start also a re-usable Flink mini cluster - Configuration flinkConfig = new Configuration(); + LOG.info("-------------------------------------------------------------------------"); + LOG.info(" KafkaTestBase finished"); + LOG.info("-------------------------------------------------------------------------"); + } + + protected static Configuration getFlinkConfiguration() { + Configuration flinkConfig = new Configuration();; flinkConfig.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1); flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 8); flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 16); flinkConfig.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "0 s"); flinkConfig.setString(ConfigConstants.METRICS_REPORTERS_LIST, "my_reporter"); flinkConfig.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "my_reporter." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, JMXReporter.class.getName()); + return flinkConfig; + } + + protected static void startClusters(boolean secureMode) throws ClassNotFoundException { + + // dynamically load the implementation for the test + Class clazz = Class.forName("org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironmentImpl"); + kafkaServer = (KafkaTestEnvironment) InstantiationUtil.instantiate(clazz); + + LOG.info("Starting KafkaTestBase.prepare() for Kafka " + kafkaServer.getVersion()); + + kafkaServer.prepare(NUMBER_OF_KAFKA_SERVERS, secureMode); + + standardProps = kafkaServer.getStandardProperties(); - flink = new LocalFlinkMiniCluster(flinkConfig, false); + brokerConnectionStrings = kafkaServer.getBrokerConnectionString(); + + if(kafkaServer.isSecureRunSupported() && secureMode) { + secureProps = kafkaServer.getSecureProperties(); + } + + // start also a re-usable Flink mini cluster + flink = new LocalFlinkMiniCluster(getFlinkConfiguration(), false); flink.start(); flinkPort = flink.getLeaderRPCPort(); - } - @AfterClass - public static void shutDownServices() { + } - LOG.info("-------------------------------------------------------------------------"); - LOG.info(" Shut down KafkaTestBase "); - LOG.info("-------------------------------------------------------------------------"); + protected static void shutdownClusters() { flinkPort = -1; if (flink != null) { flink.shutdown(); } + if(secureProps != null) { + secureProps.clear(); + } + kafkaServer.shutdown(); - LOG.info("-------------------------------------------------------------------------"); - LOG.info(" KafkaTestBase finished"); - LOG.info("-------------------------------------------------------------------------"); } @@ -164,4 +195,5 @@ protected static void createTestTopic(String topic, int numberOfPartitions, int protected static void deleteTestTopic(String topic) { kafkaServer.deleteTestTopic(topic); } + } diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java index 0b1d51d3de669..6ecde719cc6a0 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java @@ -35,10 +35,10 @@ public abstract class KafkaTestEnvironment { protected static final String KAFKA_HOST = "localhost"; - public abstract void prepare(int numKafkaServers, Properties kafkaServerProperties); + public abstract void prepare(int numKafkaServers, Properties kafkaServerProperties, boolean secureMode); - public void prepare(int numberOfKafkaServers) { - this.prepare(numberOfKafkaServers, null); + public void prepare(int numberOfKafkaServers, boolean secureMode) { + this.prepare(numberOfKafkaServers, null, secureMode); } public abstract void shutdown(); @@ -51,9 +51,10 @@ public void createTestTopic(String topic, int numberOfPartitions, int replicatio this.createTestTopic(topic, numberOfPartitions, replicationFactor, new Properties()); } - public abstract Properties getStandardProperties(); + public abstract Properties getSecureProperties(); + public abstract String getBrokerConnectionString(); public abstract String getVersion(); @@ -86,4 +87,6 @@ public FlinkKafkaConsumerBase getConsumer(String topic, DeserializationSc public abstract int getBrokerId(KafkaServer server); + public abstract boolean isSecureRunSupported(); + } diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java index 5a38e5681f54f..58a5cc307e3c5 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java @@ -135,11 +135,18 @@ public void cancel() { } }); + Properties props = new Properties(); + props.putAll(FlinkKafkaProducerBase.getPropertiesFromBrokerList(testServer.getBrokerConnectionString())); + Properties secureProps = testServer.getSecureProperties(); + if(secureProps != null) { + props.putAll(testServer.getSecureProperties()); + } + stream .rebalance() .addSink(testServer.getProducer(topic, new KeyedSerializationSchemaWrapper<>(new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, env.getConfig())), - FlinkKafkaProducerBase.getPropertiesFromBrokerList(testServer.getBrokerConnectionString()), + props, new KafkaPartitioner() { @Override public int partition(Integer next, byte[] serializedKey, byte[] serializedValue, int numPartitions) { diff --git a/flink-test-utils-parent/flink-test-utils/pom.xml b/flink-test-utils-parent/flink-test-utils/pom.xml index 18ecfded11d90..5c99ef632cc94 100644 --- a/flink-test-utils-parent/flink-test-utils/pom.xml +++ b/flink-test-utils-parent/flink-test-utils/pom.xml @@ -78,5 +78,30 @@ under the License. compile + + org.apache.hadoop + hadoop-minikdc + ${minikdc.version} + + + + + + + + + org.apache.felix + maven-bundle-plugin + 3.0.1 + true + true + + + + + diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/StreamingMultipleProgramsTestBase.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/StreamingMultipleProgramsTestBase.java index a47890877cfc9..6ec6c2cde332c 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/StreamingMultipleProgramsTestBase.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/StreamingMultipleProgramsTestBase.java @@ -25,6 +25,8 @@ import org.junit.AfterClass; import org.junit.BeforeClass; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Base class for streaming unit tests that run multiple tests and want to reuse the same @@ -67,18 +69,22 @@ public StreamingMultipleProgramsTestBase() { super(new Configuration()); } + protected static final Logger LOG = LoggerFactory.getLogger(StreamingMultipleProgramsTestBase.class); + // ------------------------------------------------------------------------ // Cluster setup & teardown // ------------------------------------------------------------------------ @BeforeClass public static void setup() throws Exception { + LOG.info("In StreamingMultipleProgramsTestBase: Starting FlinkMiniCluster "); cluster = TestBaseUtils.startCluster(1, DEFAULT_PARALLELISM, false, false, true); TestStreamEnvironment.setAsContext(cluster, DEFAULT_PARALLELISM); } @AfterClass public static void teardown() throws Exception { + LOG.info("In StreamingMultipleProgramsTestBase: Closing FlinkMiniCluster "); TestStreamEnvironment.unsetAsContext(); stopCluster(cluster, TestBaseUtils.DEFAULT_TIMEOUT); } diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/SecureTestEnvironment.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/SecureTestEnvironment.java new file mode 100644 index 0000000000000..00b19f19dce49 --- /dev/null +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/SecureTestEnvironment.java @@ -0,0 +1,249 @@ +/* + * 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.test.util; + +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.security.SecurityContext; +import org.apache.hadoop.minikdc.MiniKdc; +import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.FileWriter; +import java.io.BufferedWriter; +import java.io.PrintWriter; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +/** + * Helper {@link SecureTestEnvironment} to handle MiniKDC lifecycle. + * This class can be used to start/stop MiniKDC and create secure configurations for MiniDFSCluster + * and MiniYarn + */ + +public class SecureTestEnvironment { + + protected static final Logger LOG = LoggerFactory.getLogger(SecureTestEnvironment.class); + + private static MiniKdc kdc; + + private static String testKeytab = null; + + private static String testPrincipal = null; + + private static String testZkServerPrincipal = null; + + private static String testZkClientPrincipal = null; + + private static String testKafkaServerPrincipal = null; + + private static String hadoopServicePrincipal = null; + + private static File baseDirForSecureRun = null; + + public static void prepare(TemporaryFolder tempFolder) { + + try { + baseDirForSecureRun = tempFolder.newFolder(); + LOG.info("Base Directory for Secure Environment: {}", baseDirForSecureRun); + + String hostName = "localhost"; + Properties kdcConf = MiniKdc.createConf(); + if(LOG.isDebugEnabled()) { + kdcConf.setProperty(MiniKdc.DEBUG, "true"); + } + kdcConf.setProperty(MiniKdc.KDC_BIND_ADDRESS, hostName); + kdc = new MiniKdc(kdcConf, baseDirForSecureRun); + kdc.start(); + LOG.info("Started Mini KDC"); + + File keytabFile = new File(baseDirForSecureRun, "test-users.keytab"); + testKeytab = keytabFile.getAbsolutePath(); + testZkServerPrincipal = "zookeeper/127.0.0.1"; + testZkClientPrincipal = "zk-client/127.0.0.1"; + testKafkaServerPrincipal = "kafka/" + hostName; + hadoopServicePrincipal = "hadoop/" + hostName; + testPrincipal = "client/" + hostName; + + kdc.createPrincipal(keytabFile, testPrincipal, testZkServerPrincipal, + hadoopServicePrincipal, + testZkClientPrincipal, + testKafkaServerPrincipal); + + testPrincipal = testPrincipal + "@" + kdc.getRealm(); + testZkServerPrincipal = testZkServerPrincipal + "@" + kdc.getRealm(); + testZkClientPrincipal = testZkClientPrincipal + "@" + kdc.getRealm(); + testKafkaServerPrincipal = testKafkaServerPrincipal + "@" + kdc.getRealm(); + hadoopServicePrincipal = hadoopServicePrincipal + "@" + kdc.getRealm(); + + LOG.info("-------------------------------------------------------------------"); + LOG.info("Test Principal: {}", testPrincipal); + LOG.info("Test ZK Server Principal: {}", testZkServerPrincipal); + LOG.info("Test ZK Client Principal: {}", testZkClientPrincipal); + LOG.info("Test Kafka Server Principal: {}", testKafkaServerPrincipal); + LOG.info("Test Hadoop Service Principal: {}", hadoopServicePrincipal); + LOG.info("Test Keytab: {}", testKeytab); + LOG.info("-------------------------------------------------------------------"); + + //Security Context is established to allow non hadoop applications that requires JAAS + //based SASL/Kerberos authentication to work. However, for Hadoop specific applications + //the context can be reinitialized with Hadoop configuration by calling + //ctx.setHadoopConfiguration() for the UGI implementation to work properly. + //See Yarn test case module for reference + createJaasConfig(baseDirForSecureRun); + SecurityContext.SecurityConfiguration ctx = new SecurityContext.SecurityConfiguration(); + Configuration flinkConfig = new Configuration(); + flinkConfig.setString(ConfigConstants.SECURITY_KEYTAB_KEY, testKeytab); + flinkConfig.setString(ConfigConstants.SECURITY_PRINCIPAL_KEY, testPrincipal); + flinkConfig.setBoolean(ConfigConstants.ZOOKEEPER_SASL_DISABLE, false); + flinkConfig.setString(ConfigConstants.FLINK_BASE_DIR_PATH_KEY, baseDirForSecureRun.getAbsolutePath()); + ctx.setFlinkConfiguration(flinkConfig); + TestingSecurityContext.install(ctx, getClientSecurityConfigurationMap()); + + populateSystemEnvVariables(); + + } catch(Exception e) { + LOG.error("Exception occured while preparing secure environment. Reason: {}", e); + throw new RuntimeException(e); + } + + } + + public static void cleanup() { + + LOG.info("Cleaning up Secure Environment"); + + if( kdc != null) { + kdc.stop(); + LOG.info("Stopped KDC server"); + } + + resetSystemEnvVariables(); + + testKeytab = null; + testPrincipal = null; + testZkServerPrincipal = null; + hadoopServicePrincipal = null; + baseDirForSecureRun = null; + + } + + private static void populateSystemEnvVariables() { + + if(LOG.isDebugEnabled()) { + System.setProperty("FLINK_JAAS_DEBUG", "true"); + System.setProperty("sun.security.krb5.debug", "true"); + } + + System.setProperty("java.security.krb5.conf", kdc.getKrb5conf().getAbsolutePath()); + + System.setProperty("zookeeper.authProvider.1", "org.apache.zookeeper.server.auth.SASLAuthenticationProvider"); + System.setProperty("zookeeper.kerberos.removeHostFromPrincipal", "true"); + System.setProperty("zookeeper.kerberos.removeRealmFromPrincipal", "true"); + } + + private static void resetSystemEnvVariables() { + System.clearProperty("java.security.krb5.conf"); + System.clearProperty("FLINK_JAAS_DEBUG"); + System.clearProperty("sun.security.krb5.debug"); + + System.clearProperty("zookeeper.authProvider.1"); + System.clearProperty("zookeeper.kerberos.removeHostFromPrincipal"); + System.clearProperty("zookeeper.kerberos.removeRealmFromPrincipal"); + } + + public static org.apache.flink.configuration.Configuration populateFlinkSecureConfigurations( + @Nullable org.apache.flink.configuration.Configuration flinkConf) { + + org.apache.flink.configuration.Configuration conf; + + if(flinkConf== null) { + conf = new org.apache.flink.configuration.Configuration(); + } else { + conf = flinkConf; + } + + conf.setString(ConfigConstants.SECURITY_KEYTAB_KEY , testKeytab); + conf.setString(ConfigConstants.SECURITY_PRINCIPAL_KEY , testPrincipal); + + return conf; + } + + public static Map getClientSecurityConfigurationMap() { + + Map clientSecurityConfigurationMap = new HashMap<>(); + + if(testZkServerPrincipal != null ) { + TestingSecurityContext.ClientSecurityConfiguration zkServer = + new TestingSecurityContext.ClientSecurityConfiguration(testZkServerPrincipal, testKeytab, + "Server", "zk-server"); + clientSecurityConfigurationMap.put("Server",zkServer); + } + + if(testZkClientPrincipal != null ) { + TestingSecurityContext.ClientSecurityConfiguration zkClient = + new TestingSecurityContext.ClientSecurityConfiguration(testZkClientPrincipal, testKeytab, + "Client", "zk-client"); + clientSecurityConfigurationMap.put("Client",zkClient); + } + + if(testKafkaServerPrincipal != null ) { + TestingSecurityContext.ClientSecurityConfiguration kafkaServer = + new TestingSecurityContext.ClientSecurityConfiguration(testKafkaServerPrincipal, testKeytab, + "KafkaServer", "kafka-server"); + clientSecurityConfigurationMap.put("KafkaServer",kafkaServer); + } + + return clientSecurityConfigurationMap; + } + + public static String getTestKeytab() { + return testKeytab; + } + + public static String getHadoopServicePrincipal() { + return hadoopServicePrincipal; + } + + /* + * Helper method to create a temporary JAAS configuration file to ger around the Kafka and ZK SASL + * implementation lookup java.security.auth.login.config + */ + private static void createJaasConfig(File baseDirForSecureRun) { + + try(FileWriter fw = new FileWriter(new File(baseDirForSecureRun,SecurityContext.JAAS_CONF_FILENAME), true); + BufferedWriter bw = new BufferedWriter(fw); + PrintWriter out = new PrintWriter(bw)) + { + out.println("sample {"); + out.println("useKeyTab=false"); + out.println("useTicketCache=true;"); + out.println("};"); + } catch (IOException e) { + LOG.error("Exception occured while trying to create JAAS config. Reason: {}", e.getMessage()); + throw new RuntimeException(e); + } + + } +} diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestingJaasConfiguration.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestingJaasConfiguration.java new file mode 100644 index 0000000000000..25b236229e783 --- /dev/null +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestingJaasConfiguration.java @@ -0,0 +1,106 @@ +/* + * 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.test.util; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.security.JaasConfiguration; +import org.apache.hadoop.security.authentication.util.KerberosUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.security.auth.login.AppConfigurationEntry; +import java.util.HashMap; +import java.util.Map; + +/** + * {@link TestingJaasConfiguration} for handling the integration test case since it requires to manage + * client principal as well as server principals of Hadoop/ZK which expects the host name to be populated + * in specific way (localhost vs 127.0.0.1). This provides an abstraction to handle more than one Login Module + * since the default {@link JaasConfiguration} behavior only supports global/unique principal identifier + */ + +@Internal +public class TestingJaasConfiguration extends JaasConfiguration { + + private static final Logger LOG = LoggerFactory.getLogger(TestingJaasConfiguration.class); + + public Map clientSecurityConfigurationMap; + + TestingJaasConfiguration(String keytab, String principal, Map clientSecurityConfigurationMap) { + super(keytab, principal); + this.clientSecurityConfigurationMap = clientSecurityConfigurationMap; + } + + @Override + public AppConfigurationEntry[] getAppConfigurationEntry(String applicationName) { + + LOG.debug("In TestingJaasConfiguration - Application Requested: {}", applicationName); + + AppConfigurationEntry[] appConfigurationEntry = super.getAppConfigurationEntry(applicationName); + + if(clientSecurityConfigurationMap != null && clientSecurityConfigurationMap.size() > 0) { + + if(clientSecurityConfigurationMap.containsKey(applicationName)) { + + LOG.debug("In TestingJaasConfiguration - Application: {} found in the supplied context", applicationName); + + TestingSecurityContext.ClientSecurityConfiguration conf = clientSecurityConfigurationMap.get(applicationName); + + if(appConfigurationEntry != null && appConfigurationEntry.length > 0) { + + for(int count=0; count < appConfigurationEntry.length; count++) { + + AppConfigurationEntry ace = appConfigurationEntry[count]; + + if (ace.getOptions().containsKey("keyTab")) { + + String keyTab = conf.getKeytab(); + String principal = conf.getPrincipal(); + + LOG.debug("In TestingJaasConfiguration - Application: {} from the supplied context will " + + "use Client Specific Keytab: {} and Principal: {}", applicationName, keyTab, principal); + + Map newKeytabKerberosOptions = new HashMap<>(); + newKeytabKerberosOptions.putAll(getKeytabKerberosOptions()); + + newKeytabKerberosOptions.put("keyTab", keyTab); + newKeytabKerberosOptions.put("principal", principal); + + AppConfigurationEntry keytabKerberosAce = new AppConfigurationEntry( + KerberosUtil.getKrb5LoginModuleName(), + AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, + newKeytabKerberosOptions); + appConfigurationEntry = new AppConfigurationEntry[] {keytabKerberosAce}; + + LOG.debug("---->Login Module is using Keytab based configuration<------"); + LOG.debug("Login Module Name: " + keytabKerberosAce.getLoginModuleName()); + LOG.debug("Control Flag: " + keytabKerberosAce.getControlFlag()); + LOG.debug("Options: " + keytabKerberosAce.getOptions()); + } + } + } + } + + } + + return appConfigurationEntry; + } + +} diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestingSecurityContext.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestingSecurityContext.java new file mode 100644 index 0000000000000..5e84c7eebf32d --- /dev/null +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestingSecurityContext.java @@ -0,0 +1,80 @@ +/* + * 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.test.util; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.security.SecurityContext; + +import java.util.Map; + +/* + * Test security context to support handling both client and server principals in MiniKDC + * This class is used only in integration test code for connectors like Kafka, HDFS etc., + */ +@Internal +public class TestingSecurityContext { + + public static void install(SecurityContext.SecurityConfiguration config, + Map clientSecurityConfigurationMap) + throws Exception { + + SecurityContext.install(config); + + // establish the JAAS config for Test environment + TestingJaasConfiguration jaasConfig = new TestingJaasConfiguration(config.getKeytab(), + config.getPrincipal(), clientSecurityConfigurationMap); + javax.security.auth.login.Configuration.setConfiguration(jaasConfig); + } + + public static class ClientSecurityConfiguration { + + private String principal; + + private String keytab; + + private String moduleName; + + private String jaasServiceName; + + public String getPrincipal() { + return principal; + } + + public String getKeytab() { + return keytab; + } + + public String getModuleName() { + return moduleName; + } + + public String getJaasServiceName() { + return jaasServiceName; + } + + public ClientSecurityConfiguration(String principal, String keytab, String moduleName, String jaasServiceName) { + this.principal = principal; + this.keytab = keytab; + this.moduleName = moduleName; + this.jaasServiceName = jaasServiceName; + } + + } + +} \ No newline at end of file diff --git a/flink-yarn-tests/pom.xml b/flink-yarn-tests/pom.xml index ffdca36264b9d..68e4752f789bb 100644 --- a/flink-yarn-tests/pom.xml +++ b/flink-yarn-tests/pom.xml @@ -103,6 +103,13 @@ under the License. akka-testkit_${scala.binary.version} test + + + org.apache.hadoop + hadoop-minikdc + ${minikdc.version} + + @@ -298,6 +305,19 @@ under the License. true + + + + org.apache.felix + maven-bundle-plugin + 3.0.1 + true + true + + diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java index d03d9eb4a4dec..a503115125570 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java @@ -180,7 +180,7 @@ public TestingYarnClusterClient(AbstractYarnClusterDescriptor descriptor, Config Mockito.mock(YarnClient.class), Mockito.mock(ApplicationReport.class), config, - new Path("/tmp"), false); + new Path("/temp"), false); } } } diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java index a2933482de159..9d6ff85bf8e8f 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java @@ -50,14 +50,14 @@ public class YARNHighAvailabilityITCase extends YarnTestBase { - private static TestingServer zkServer; + protected static TestingServer zkServer; - private static ActorSystem actorSystem; + protected static ActorSystem actorSystem; - private static final int numberApplicationAttempts = 10; + protected static final int numberApplicationAttempts = 10; @Rule - public TemporaryFolder tmp = new TemporaryFolder(); + public TemporaryFolder temp = new TemporaryFolder(); @BeforeClass public static void setup() { @@ -108,7 +108,11 @@ public void testMultipleAMKill() throws Exception { String confDirPath = System.getenv(ConfigConstants.ENV_FLINK_CONF_DIR); flinkYarnClient.setConfigurationDirectory(confDirPath); - String fsStateHandlePath = tmp.getRoot().getPath(); + String fsStateHandlePath = temp.getRoot().getPath(); + + // load the configuration + File configDirectory = new File(confDirPath); + GlobalConfiguration.loadConfiguration(configDirectory.getAbsolutePath()); flinkYarnClient.setFlinkConfiguration(GlobalConfiguration.loadConfiguration()); flinkYarnClient.setDynamicPropertiesEncoded("recovery.mode=zookeeper@@recovery.zookeeper.quorum=" + diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java index ddea4ddea0bd0..650397dfabcb1 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java @@ -23,10 +23,12 @@ import com.google.common.base.Joiner; import org.apache.commons.io.FileUtils; import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.runtime.client.JobClient; import org.apache.flink.runtime.webmonitor.WebMonitorUtils; import org.apache.flink.test.testdata.WordCountData; import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.yarn.cli.FlinkYarnSessionCli; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest; import org.apache.hadoop.yarn.api.records.ApplicationId; @@ -516,6 +518,27 @@ public boolean accept(File dir, String name) { } catch(Throwable t) { LOG.warn("Error while detached yarn session was running", t); Assert.fail(t.getMessage()); + } finally { + + //cleanup the yarn-properties file + String confDirPath = System.getenv("FLINK_CONF_DIR"); + File configDirectory = new File(confDirPath); + LOG.info("testDetachedPerJobYarnClusterInternal: Using configuration directory " + configDirectory.getAbsolutePath()); + + // load the configuration + LOG.info("testDetachedPerJobYarnClusterInternal: Trying to load configuration file"); + GlobalConfiguration.loadConfiguration(configDirectory.getAbsolutePath()); + + try { + File yarnPropertiesFile = FlinkYarnSessionCli.getYarnPropertiesLocation(GlobalConfiguration.loadConfiguration()); + if(yarnPropertiesFile.exists()) { + LOG.info("testDetachedPerJobYarnClusterInternal: Cleaning up temporary Yarn address reference: {}", yarnPropertiesFile.getAbsolutePath()); + yarnPropertiesFile.delete(); + } + } catch (Exception e) { + LOG.warn("testDetachedPerJobYarnClusterInternal: Exception while deleting the JobManager address file", e); + } + } } diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java index 3caa0ee2cdaf3..ca696f9cf1404 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java @@ -100,6 +100,9 @@ public void testDetachedMode() { while(getRunningContainers() < 2) { sleep(500); } + + //additional sleep for the JM/TM to start and establish connection + sleep(2000); LOG.info("Two containers are running. Killing the application"); // kill application "externally". @@ -121,6 +124,27 @@ public void testDetachedMode() { } catch(Throwable t) { LOG.warn("Killing failed", t); Assert.fail(); + } finally { + + //cleanup the yarn-properties file + String confDirPath = System.getenv("FLINK_CONF_DIR"); + File configDirectory = new File(confDirPath); + LOG.info("testDetachedPerJobYarnClusterInternal: Using configuration directory " + configDirectory.getAbsolutePath()); + + // load the configuration + LOG.info("testDetachedPerJobYarnClusterInternal: Trying to load configuration file"); + GlobalConfiguration.loadConfiguration(configDirectory.getAbsolutePath()); + + try { + File yarnPropertiesFile = FlinkYarnSessionCli.getYarnPropertiesLocation(GlobalConfiguration.loadConfiguration()); + if(yarnPropertiesFile.exists()) { + LOG.info("testDetachedPerJobYarnClusterInternal: Cleaning up temporary Yarn address reference: {}", yarnPropertiesFile.getAbsolutePath()); + yarnPropertiesFile.delete(); + } + } catch (Exception e) { + LOG.warn("testDetachedPerJobYarnClusterInternal: Exception while deleting the JobManager address file", e); + } + } LOG.info("Finished testDetachedMode()"); diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java new file mode 100644 index 0000000000000..0b7c2303cbdaf --- /dev/null +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java @@ -0,0 +1,103 @@ +/** + * 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.yarn; + +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.security.SecurityContext; +import org.apache.flink.test.util.SecureTestEnvironment; +import org.apache.flink.test.util.TestingSecurityContext; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class YARNSessionFIFOSecuredITCase extends YARNSessionFIFOITCase { + + protected static final Logger LOG = LoggerFactory.getLogger(YARNSessionFIFOSecuredITCase.class); + + @BeforeClass + public static void setup() { + + LOG.info("starting secure cluster environment for testing"); + + yarnConfiguration.setClass(YarnConfiguration.RM_SCHEDULER, FifoScheduler.class, ResourceScheduler.class); + yarnConfiguration.setInt(YarnConfiguration.NM_PMEM_MB, 768); + yarnConfiguration.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 512); + yarnConfiguration.set(YarnTestBase.TEST_CLUSTER_NAME_KEY, "flink-yarn-tests-fifo-secured"); + + SecureTestEnvironment.prepare(tmp); + + populateYarnSecureConfigurations(yarnConfiguration,SecureTestEnvironment.getHadoopServicePrincipal(), + SecureTestEnvironment.getTestKeytab()); + + Configuration flinkConfig = new Configuration(); + flinkConfig.setString(ConfigConstants.SECURITY_KEYTAB_KEY, + SecureTestEnvironment.getTestKeytab()); + flinkConfig.setString(ConfigConstants.SECURITY_PRINCIPAL_KEY, + SecureTestEnvironment.getHadoopServicePrincipal()); + + SecurityContext.SecurityConfiguration ctx = new SecurityContext.SecurityConfiguration(); + ctx.setFlinkConfiguration(flinkConfig); + ctx.setHadoopConfiguration(yarnConfiguration); + try { + TestingSecurityContext.install(ctx, SecureTestEnvironment.getClientSecurityConfigurationMap()); + + SecurityContext.getInstalled().runSecured(new SecurityContext.FlinkSecuredRunner() { + @Override + public Integer run() { + startYARNSecureMode(yarnConfiguration, SecureTestEnvironment.getHadoopServicePrincipal(), + SecureTestEnvironment.getTestKeytab()); + return null; + } + }); + + } catch(Exception e) { + throw new RuntimeException("Exception occurred while setting up secure test context. Reason: {}", e); + } + + } + + @AfterClass + public static void teardownSecureCluster() throws Exception { + LOG.info("tearing down secure cluster environment"); + SecureTestEnvironment.cleanup(); + } + + /* For secure cluster testing, it is enough to run only one test and override below test methods + * to keep the overall build time minimal + */ + @Override + public void testQueryCluster() {} + + @Override + public void testNonexistingQueue() {} + + @Override + public void testResourceComputation() {} + + @Override + public void testfullAlloc() {} + + @Override + public void testJavaAPI() {} +} diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java index 627001084678b..605aa443fe785 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java @@ -20,6 +20,7 @@ import akka.actor.Identify; import org.apache.commons.io.FileUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.client.CliFrontend; import org.apache.flink.client.cli.CommandLineOptions; import org.apache.flink.client.program.ClusterClient; @@ -29,6 +30,8 @@ import org.apache.flink.test.util.TestBaseUtils; import org.apache.flink.util.TestLogger; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.service.Service; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.ContainerId; @@ -62,6 +65,8 @@ import java.io.FilenameFilter; import java.io.IOException; import java.io.PrintStream; +import java.io.BufferedWriter; +import java.io.PrintWriter; import java.util.Arrays; import java.util.HashMap; import java.util.List; @@ -123,6 +128,11 @@ public abstract class YarnTestBase extends TestLogger { */ protected static File flinkLibFolder; + /** + * Temporary folder where Flink configurations will be kept for secure run + */ + protected static File tempConfPathForSecureRun = null; + static { yarnConfiguration = new YarnConfiguration(); yarnConfiguration.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 512); @@ -140,6 +150,23 @@ public abstract class YarnTestBase extends TestLogger { } + public static void populateYarnSecureConfigurations(Configuration conf, String principal, String keytab) { + + conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos"); + conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION, "true"); + + conf.set(YarnConfiguration.RM_KEYTAB, keytab); + conf.set(YarnConfiguration.RM_PRINCIPAL, principal); + conf.set(YarnConfiguration.NM_KEYTAB, keytab); + conf.set(YarnConfiguration.NM_PRINCIPAL, principal); + + conf.set(YarnConfiguration.RM_WEBAPP_SPNEGO_USER_NAME_KEY, principal); + conf.set(YarnConfiguration.RM_WEBAPP_SPNEGO_KEYTAB_FILE_KEY,keytab); + conf.set(YarnConfiguration.NM_WEBAPP_SPNEGO_USER_NAME_KEY, principal); + conf.set(YarnConfiguration.NM_WEBAPP_SPNEGO_KEYTAB_FILE_KEY,keytab); + + conf.set("hadoop.security.auth_to_local","RULE:[1:$1] RULE:[2:$1]"); + } /** * Sleep a bit between the tests (we are re-using the YARN cluster for the tests) @@ -336,8 +363,16 @@ public static int getRunningContainers() { return count; } + public static void startYARNSecureMode(Configuration conf, String principal, String keytab) { + start(conf, principal, keytab); + } + public static void startYARNWithConfig(Configuration conf) { - // set the home directory to a tmp directory. Flink on YARN is using the home dir to distribute the file + start(conf,null,null); + } + + private static void start(Configuration conf, String principal, String keytab) { + // set the home directory to a temp directory. Flink on YARN is using the home dir to distribute the file File homeDir = null; try { homeDir = tmp.newFolder(); @@ -374,7 +409,39 @@ public static void startYARNWithConfig(Configuration conf) { File flinkConfDirPath = findFile(flinkDistRootDir, new ContainsName(new String[]{"flink-conf.yaml"})); Assert.assertNotNull(flinkConfDirPath); - map.put(ConfigConstants.ENV_FLINK_CONF_DIR, flinkConfDirPath.getParent()); + if(!StringUtils.isBlank(principal) && !StringUtils.isBlank(keytab)) { + //copy conf dir to test temporary workspace location + tempConfPathForSecureRun = tmp.newFolder("conf"); + + String confDirPath = flinkConfDirPath.getParentFile().getAbsolutePath(); + FileUtils.copyDirectory(new File(confDirPath), tempConfPathForSecureRun); + + try(FileWriter fw = new FileWriter(new File(tempConfPathForSecureRun,"flink-conf.yaml"), true); + BufferedWriter bw = new BufferedWriter(fw); + PrintWriter out = new PrintWriter(bw)) + { + LOG.info("writing keytab: " + keytab + " and principal: " + principal + " to config file"); + out.println(""); + out.println("#Security Configurations Auto Populated "); + out.println(ConfigConstants.SECURITY_KEYTAB_KEY + ": " + keytab); + out.println(ConfigConstants.SECURITY_PRINCIPAL_KEY + ": " + principal); + out.println(""); + } catch (IOException e) { + LOG.error("Exception occured while trying to append the security configurations. Reason: {}", e.getMessage()); + throw new RuntimeException(e); + } + + String configDir = tempConfPathForSecureRun.getAbsolutePath(); + + LOG.info("Temporary Flink configuration directory to be used for secure test: {}", configDir); + + Assert.assertNotNull(configDir); + + map.put(ConfigConstants.ENV_FLINK_CONF_DIR, configDir); + + } else { + map.put(ConfigConstants.ENV_FLINK_CONF_DIR, flinkConfDirPath.getParent()); + } File yarnConfFile = writeYarnSiteConfigXML(conf); map.put("YARN_CONF_DIR", yarnConfFile.getParentFile().getAbsolutePath()); @@ -392,6 +459,7 @@ public static void startYARNWithConfig(Configuration conf) { LOG.error("setup failure", ex); Assert.fail(); } + } /** @@ -421,7 +489,6 @@ protected Runner startWithArgs(String[] args, String startedAfterString, RunType System.setOut(new PrintStream(outContent)); System.setErr(new PrintStream(errContent)); - final int START_TIMEOUT_SECONDS = 60; Runner runner = new Runner(args, type); @@ -624,12 +691,23 @@ public void sendStop() { @AfterClass public static void teardown() throws Exception { + + LOG.info("Stopping MiniYarn Cluster"); + yarnCluster.stop(); + // Unset FLINK_CONF_DIR, as it might change the behavior of other tests Map map = new HashMap<>(System.getenv()); map.remove(ConfigConstants.ENV_FLINK_CONF_DIR); + map.remove("YARN_CONF_DIR"); + map.remove("IN_TESTS"); TestBaseUtils.setEnv(map); - // When we are on travis, we copy the tmp files of JUnit (containing the MiniYARNCluster log files) + if(tempConfPathForSecureRun != null) { + FileUtil.fullyDelete(tempConfPathForSecureRun); + tempConfPathForSecureRun = null; + } + + // When we are on travis, we copy the temp files of JUnit (containing the MiniYARNCluster log files) // to /target/flink-yarn-tests-*. // The files from there are picked up by the ./tools/travis_watchdog.sh script // to upload them to Amazon S3. @@ -646,6 +724,7 @@ public static void teardown() throws Exception { LOG.warn("Error copying the final files from {} to {}: msg: {}", src.getAbsolutePath(), target.getAbsolutePath(), e.getMessage(), e); } } + } public static boolean isOnTravis() { diff --git a/flink-yarn-tests/src/test/resources/log4j-test.properties b/flink-yarn-tests/src/test/resources/log4j-test.properties index e94ca26fc06d5..8f56c1fa9a55c 100644 --- a/flink-yarn-tests/src/test/resources/log4j-test.properties +++ b/flink-yarn-tests/src/test/resources/log4j-test.properties @@ -34,3 +34,8 @@ log4j.logger.org.apache.hadoop=OFF log4j.logger.org.apache.flink.runtime.leaderelection=INFO log4j.logger.org.apache.flink.runtime.leaderretrieval=INFO +log4j.logger.org.apache.directory=OFF +log4j.logger.org.mortbay.log=OFF, testlogger +log4j.logger.net.sf.ehcache=OFF +log4j.logger.org.apache.hadoop.metrics2=OFF +log4j.logger.org.apache.hadoop.conf.Configuration=OFF diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java index f4c20323466eb..848013c5eaccf 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java @@ -24,6 +24,7 @@ import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; +import org.apache.flink.runtime.security.SecurityContext; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -59,7 +60,6 @@ import java.io.PrintStream; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; -import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -341,26 +341,8 @@ public YarnClusterClient retrieve(String applicationID) { @Override public YarnClusterClient deploy() { - try { - - UserGroupInformation.setConfiguration(conf); - UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); - - if (UserGroupInformation.isSecurityEnabled()) { - if (!ugi.hasKerberosCredentials()) { - throw new YarnDeploymentException("In secure mode. Please provide Kerberos credentials in order to authenticate. " + - "You may use kinit to authenticate and request a TGT from the Kerberos server."); - } - return ugi.doAs(new PrivilegedExceptionAction() { - @Override - public YarnClusterClient run() throws Exception { - return deployInternal(); - } - }); - } else { - return deployInternal(); - } + return deployInternal(); } catch (Exception e) { throw new RuntimeException("Couldn't deploy Yarn cluster", e); } @@ -539,9 +521,13 @@ protected YarnClusterClient deployInternal() throws Exception { } } - addLibFolderToShipFiles(effectiveShipFiles); + //check if there is a JAAS config file + File jaasConfigFile = new File(configurationDirectory + File.separator + SecurityContext.JAAS_CONF_FILENAME); + if (jaasConfigFile.exists() && jaasConfigFile.isFile()) { + effectiveShipFiles.add(jaasConfigFile); + } - final ContainerLaunchContext amContainer = setupApplicationMasterContainer(hasLogback, hasLog4j); + addLibFolderToShipFiles(effectiveShipFiles); // Set-up ApplicationSubmissionContext for the application ApplicationSubmissionContext appContext = yarnApplication.getApplicationSubmissionContext(); @@ -626,8 +612,53 @@ protected YarnClusterClient deployInternal() throws Exception { FsPermission permission = new FsPermission(FsAction.ALL, FsAction.NONE, FsAction.NONE); fs.setPermission(sessionFilesDir, permission); // set permission for path. + //To support Yarn Secure Integration Test Scenario + //In Integration test setup, the Yarn containers created by YarnMiniCluster does not have the Yarn site XML + //and KRB5 configuration files. We are adding these files as container local resources for the container + //applications (JM/TMs) to have proper secure cluster setup + Path remoteKrb5Path = null; + Path remoteYarnSiteXmlPath = null; + boolean hasKrb5 = false; + if(System.getenv("IN_TESTS") != null) { + String krb5Config = System.getProperty("java.security.krb5.conf"); + if(krb5Config != null && krb5Config.length() != 0) { + File krb5 = new File(krb5Config); + LOG.info("Adding KRB5 configuration {} to the AM container local resource bucket", krb5.getAbsolutePath()); + LocalResource krb5ConfResource = Records.newRecord(LocalResource.class); + Path krb5ConfPath = new Path(krb5.getAbsolutePath()); + remoteKrb5Path = Utils.setupLocalResource(fs, appId.toString(), krb5ConfPath, krb5ConfResource, fs.getHomeDirectory()); + localResources.put(Utils.KRB5_FILE_NAME, krb5ConfResource); + + File f = new File(System.getenv("YARN_CONF_DIR"),Utils.YARN_SITE_FILE_NAME); + LOG.info("Adding Yarn configuration {} to the AM container local resource bucket", f.getAbsolutePath()); + LocalResource yarnConfResource = Records.newRecord(LocalResource.class); + Path yarnSitePath = new Path(f.getAbsolutePath()); + remoteYarnSiteXmlPath = Utils.setupLocalResource(fs, appId.toString(), yarnSitePath, yarnConfResource, fs.getHomeDirectory()); + localResources.put(Utils.YARN_SITE_FILE_NAME, yarnConfResource); + + hasKrb5 = true; + } + } + // setup security tokens - Utils.setTokensFor(amContainer, paths, conf); + LocalResource keytabResource = null; + Path remotePathKeytab = null; + String keytab = flinkConfiguration.getString(ConfigConstants.SECURITY_KEYTAB_KEY, null); + if(keytab != null) { + LOG.info("Adding keytab {} to the AM container local resource bucket", keytab); + keytabResource = Records.newRecord(LocalResource.class); + Path keytabPath = new Path(keytab); + remotePathKeytab = Utils.setupLocalResource(fs, appId.toString(), keytabPath, keytabResource, fs.getHomeDirectory()); + localResources.put(Utils.KEYTAB_FILE_NAME, keytabResource); + } + + final ContainerLaunchContext amContainer = setupApplicationMasterContainer(hasLogback, hasLog4j, hasKrb5); + + if ( UserGroupInformation.isSecurityEnabled() && keytab == null ) { + //set tokens only when keytab is not provided + LOG.info("Adding delegation token to the AM container.."); + Utils.setTokensFor(amContainer, paths, conf); + } amContainer.setLocalResources(localResources); fs.close(); @@ -646,11 +677,25 @@ protected YarnClusterClient deployInternal() throws Exception { appMasterEnv.put(YarnConfigKeys.ENV_APP_ID, appId.toString()); appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_HOME_DIR, fs.getHomeDirectory().toString()); appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_SHIP_FILES, envShipFileList.toString()); - appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_USERNAME, UserGroupInformation.getCurrentUser().getShortUserName()); appMasterEnv.put(YarnConfigKeys.ENV_SLOTS, String.valueOf(slots)); appMasterEnv.put(YarnConfigKeys.ENV_DETACHED, String.valueOf(detached)); appMasterEnv.put(YarnConfigKeys.ENV_ZOOKEEPER_NAMESPACE, getZookeeperNamespace()); + // https://github.com/apache/hadoop/blob/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnApplicationSecurity.md#identity-on-an-insecure-cluster-hadoop_user_name + appMasterEnv.put(YarnConfigKeys.ENV_HADOOP_USER_NAME, UserGroupInformation.getCurrentUser().getUserName()); + + if(keytabResource != null) { + appMasterEnv.put(YarnConfigKeys.KEYTAB_PATH, remotePathKeytab.toString() ); + String principal = flinkConfiguration.getString(ConfigConstants.SECURITY_PRINCIPAL_KEY, null); + appMasterEnv.put(YarnConfigKeys.KEYTAB_PRINCIPAL, principal ); + } + + //To support Yarn Secure Integration Test Scenario + if(remoteYarnSiteXmlPath != null && remoteKrb5Path != null) { + appMasterEnv.put(YarnConfigKeys.ENV_YARN_SITE_XML_PATH, remoteYarnSiteXmlPath.toString()); + appMasterEnv.put(YarnConfigKeys.ENV_KRB5_PATH, remoteKrb5Path.toString() ); + } + if(dynamicPropertiesEncoded != null) { appMasterEnv.put(YarnConfigKeys.ENV_DYNAMIC_PROPERTIES, dynamicPropertiesEncoded); } @@ -700,6 +745,7 @@ protected YarnClusterClient deployInternal() throws Exception { throw new YarnDeploymentException("Failed to deploy the cluster: " + e.getMessage()); } YarnApplicationState appState = report.getYarnApplicationState(); + LOG.debug("Application State: {}", appState); switch(appState) { case FAILED: case FINISHED: @@ -996,7 +1042,9 @@ protected void addLibFolderToShipFiles(Set effectiveShipFiles) { } } - protected ContainerLaunchContext setupApplicationMasterContainer(boolean hasLogback, boolean hasLog4j) { + protected ContainerLaunchContext setupApplicationMasterContainer(boolean hasLogback, + boolean hasLog4j, + boolean hasKrb5) { // ------------------ Prepare Application Master Container ------------------------------ // respect custom JVM options in the YAML file @@ -1021,6 +1069,12 @@ protected ContainerLaunchContext setupApplicationMasterContainer(boolean hasLogb } } + //applicable only for YarnMiniCluster secure test run + //krb5.conf file will be available as local resource in JM/TM container + if(hasKrb5) { + amCommand += " -Djava.security.krb5.conf=krb5.conf"; + } + amCommand += " " + getApplicationMasterClass().getName() + " " + " 1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/jobmanager.out" diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java index 1496d61c90e19..94d4582ad62a4 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java @@ -60,6 +60,14 @@ public final class Utils { private static final Logger LOG = LoggerFactory.getLogger(Utils.class); + /** Keytab file name populated in YARN container */ + public static final String KEYTAB_FILE_NAME = "krb5.keytab"; + + /** KRB5 file name populated in YARN container for secure IT run */ + public static final String KRB5_FILE_NAME = "krb5.conf"; + + /** Yarn site xml file name populated in YARN container for secure IT run */ + public static final String YARN_SITE_FILE_NAME = "yarn-site.xml"; /** * See documentation diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java index 6619633eb14db..efb658a1b6c3b 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java @@ -32,6 +32,7 @@ import org.apache.flink.runtime.jobmanager.MemoryArchivist; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.process.ProcessReaper; +import org.apache.flink.runtime.security.SecurityContext; import org.apache.flink.runtime.taskmanager.TaskManager; import org.apache.flink.runtime.util.EnvironmentInformation; import org.apache.flink.runtime.util.JvmShutdownSafeguard; @@ -40,11 +41,11 @@ import org.apache.flink.runtime.webmonitor.WebMonitor; import org.apache.flink.yarn.cli.FlinkYarnSessionCli; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.security.token.Token; import org.apache.hadoop.yarn.api.ApplicationConstants; import org.apache.hadoop.yarn.api.ApplicationConstants.Environment; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; @@ -60,11 +61,10 @@ import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; -import java.security.PrivilegedAction; -import java.util.Collections; -import java.util.HashMap; import java.util.Map; +import java.util.HashMap; import java.util.UUID; +import java.util.Collections; import java.util.concurrent.TimeUnit; import static org.apache.flink.yarn.YarnConfigKeys.ENV_FLINK_CLASSPATH; @@ -117,7 +117,7 @@ public static void main(String[] args) { /** * The instance entry point for the YARN application master. Obtains user group - * information and calls the main work method {@link #runApplicationMaster()} as a + * information and calls the main work method {@link #runApplicationMaster(Configuration)} as a * privileged action. * * @param args The command line arguments. @@ -127,34 +127,66 @@ protected int run(String[] args) { try { LOG.debug("All environment variables: {}", ENV); - final String yarnClientUsername = ENV.get(YarnConfigKeys.ENV_CLIENT_USERNAME); + final String yarnClientUsername = ENV.get(YarnConfigKeys.ENV_HADOOP_USER_NAME); require(yarnClientUsername != null, "YARN client user name environment variable {} not set", - YarnConfigKeys.ENV_CLIENT_USERNAME); + YarnConfigKeys.ENV_HADOOP_USER_NAME); - final UserGroupInformation currentUser; - try { - currentUser = UserGroupInformation.getCurrentUser(); - } catch (Throwable t) { - throw new Exception("Cannot access UserGroupInformation information for current user", t); + final String currDir = ENV.get(Environment.PWD.key()); + require(currDir != null, "Current working directory variable (%s) not set", Environment.PWD.key()); + LOG.debug("Current working Directory: {}", currDir); + + final String remoteKeytabPath = ENV.get(YarnConfigKeys.KEYTAB_PATH); + LOG.debug("remoteKeytabPath obtained {}", remoteKeytabPath); + + final String remoteKeytabPrincipal = ENV.get(YarnConfigKeys.KEYTAB_PRINCIPAL); + LOG.info("remoteKeytabPrincipal obtained {}", remoteKeytabPrincipal); + + String keytabPath = null; + if(remoteKeytabPath != null) { + File f = new File(currDir, Utils.KEYTAB_FILE_NAME); + keytabPath = f.getAbsolutePath(); + LOG.debug("keytabPath: {}", keytabPath); } - LOG.info("YARN daemon runs as user {}. Running Flink Application Master/JobManager as user {}", - currentUser.getShortUserName(), yarnClientUsername); + UserGroupInformation currentUser = UserGroupInformation.getCurrentUser(); + + LOG.info("YARN daemon is running as: {} Yarn client user obtainer: {}", + currentUser.getShortUserName(), yarnClientUsername ); + + SecurityContext.SecurityConfiguration sc = new SecurityContext.SecurityConfiguration(); + + //To support Yarn Secure Integration Test Scenario + File krb5Conf = new File(currDir, Utils.KRB5_FILE_NAME); + if(krb5Conf.exists() && krb5Conf.canRead()) { + String krb5Path = krb5Conf.getAbsolutePath(); + LOG.info("KRB5 Conf: {}", krb5Path); + org.apache.hadoop.conf.Configuration conf = new org.apache.hadoop.conf.Configuration(); + conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos"); + conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION, "true"); + sc.setHadoopConfiguration(conf); + } - UserGroupInformation ugi = UserGroupInformation.createRemoteUser(yarnClientUsername); + // Flink configuration + final Map dynamicProperties = + FlinkYarnSessionCli.getDynamicProperties(ENV.get(YarnConfigKeys.ENV_DYNAMIC_PROPERTIES)); + LOG.debug("YARN dynamic properties: {}", dynamicProperties); - // transfer all security tokens, for example for authenticated HDFS and HBase access - for (Token token : currentUser.getTokens()) { - ugi.addToken(token); + final Configuration flinkConfig = createConfiguration(currDir, dynamicProperties); + if(keytabPath != null && remoteKeytabPrincipal != null) { + flinkConfig.setString(ConfigConstants.SECURITY_KEYTAB_KEY, keytabPath); + flinkConfig.setString(ConfigConstants.SECURITY_PRINCIPAL_KEY, remoteKeytabPrincipal); } + flinkConfig.setString(ConfigConstants.FLINK_BASE_DIR_PATH_KEY, currDir); - // run the actual work in a secured privileged action - return ugi.doAs(new PrivilegedAction() { + SecurityContext.install(sc.setFlinkConfiguration(flinkConfig)); + + return SecurityContext.getInstalled().runSecured(new SecurityContext.FlinkSecuredRunner() { @Override public Integer run() { - return runApplicationMaster(); + return runApplicationMaster(flinkConfig); } }); + } catch (Throwable t) { // make sure that everything whatever ends up in the log @@ -172,7 +204,7 @@ public Integer run() { * * @return The return code for the Java process. */ - protected int runApplicationMaster() { + protected int runApplicationMaster(Configuration config) { ActorSystem actorSystem = null; WebMonitor webMonitor = null; @@ -194,12 +226,21 @@ protected int runApplicationMaster() { LOG.info("YARN assigned hostname for application master: {}", appMasterHostname); - // Flink configuration - final Map dynamicProperties = - FlinkYarnSessionCli.getDynamicProperties(ENV.get(YarnConfigKeys.ENV_DYNAMIC_PROPERTIES)); - LOG.debug("YARN dynamic properties: {}", dynamicProperties); + //Update keytab and principal path to reflect YARN container path location + final String remoteKeytabPath = ENV.get(YarnConfigKeys.KEYTAB_PATH); - final Configuration config = createConfiguration(currDir, dynamicProperties); + final String remoteKeytabPrincipal = ENV.get(YarnConfigKeys.KEYTAB_PRINCIPAL); + + String keytabPath = null; + if(remoteKeytabPath != null) { + File f = new File(currDir, Utils.KEYTAB_FILE_NAME); + keytabPath = f.getAbsolutePath(); + LOG.info("keytabPath: {}", keytabPath); + } + if(keytabPath != null && remoteKeytabPrincipal != null) { + config.setString(ConfigConstants.SECURITY_KEYTAB_KEY, keytabPath); + config.setString(ConfigConstants.SECURITY_PRINCIPAL_KEY, remoteKeytabPrincipal); + } // Hadoop/Yarn configuration (loads config data automatically from classpath files) final YarnConfiguration yarnConfig = new YarnConfiguration(); @@ -523,8 +564,20 @@ public static ContainerLaunchContext createTaskManagerContext( String shipListString = env.get(YarnConfigKeys.ENV_CLIENT_SHIP_FILES); require(shipListString != null, "Environment variable %s not set", YarnConfigKeys.ENV_CLIENT_SHIP_FILES); - String yarnClientUsername = env.get(YarnConfigKeys.ENV_CLIENT_USERNAME); - require(yarnClientUsername != null, "Environment variable %s not set", YarnConfigKeys.ENV_CLIENT_USERNAME); + String yarnClientUsername = env.get(YarnConfigKeys.ENV_HADOOP_USER_NAME); + require(yarnClientUsername != null, "Environment variable %s not set", YarnConfigKeys.ENV_HADOOP_USER_NAME); + + final String remoteKeytabPath = env.get(YarnConfigKeys.KEYTAB_PATH); + LOG.info("TM:remoteKeytabPath obtained {}", remoteKeytabPath); + + final String remoteKeytabPrincipal = env.get(YarnConfigKeys.KEYTAB_PRINCIPAL); + LOG.info("TM:remoteKeytabPrincipal obtained {}", remoteKeytabPrincipal); + + final String remoteYarnConfPath = env.get(YarnConfigKeys.ENV_YARN_SITE_XML_PATH); + LOG.info("TM:remoteYarnConfPath obtained {}", remoteYarnConfPath); + + final String remoteKrb5Path = env.get(YarnConfigKeys.ENV_KRB5_PATH); + LOG.info("TM:remotekrb5Path obtained {}", remoteKrb5Path); String classPathString = env.get(YarnConfigKeys.ENV_FLINK_CLASSPATH); require(classPathString != null, "Environment variable %s not set", YarnConfigKeys.ENV_FLINK_CLASSPATH); @@ -537,6 +590,33 @@ public static ContainerLaunchContext createTaskManagerContext( throw new Exception("Could not access YARN's default file system", e); } + //register keytab + LocalResource keytabResource = null; + if(remoteKeytabPath != null) { + LOG.info("Adding keytab {} to the AM container local resource bucket", remoteKeytabPath); + keytabResource = Records.newRecord(LocalResource.class); + Path keytabPath = new Path(remoteKeytabPath); + Utils.registerLocalResource(yarnFileSystem, keytabPath, keytabResource); + } + + //To support Yarn Secure Integration Test Scenario + LocalResource yarnConfResource = null; + LocalResource krb5ConfResource = null; + boolean hasKrb5 = false; + if(remoteYarnConfPath != null && remoteKrb5Path != null) { + LOG.info("TM:Adding remoteYarnConfPath {} to the container local resource bucket", remoteYarnConfPath); + yarnConfResource = Records.newRecord(LocalResource.class); + Path yarnConfPath = new Path(remoteYarnConfPath); + Utils.registerLocalResource(yarnFileSystem, yarnConfPath, yarnConfResource); + + LOG.info("TM:Adding remoteKrb5Path {} to the container local resource bucket", remoteKrb5Path); + krb5ConfResource = Records.newRecord(LocalResource.class); + Path krb5ConfPath = new Path(remoteKrb5Path); + Utils.registerLocalResource(yarnFileSystem, krb5ConfPath, krb5ConfResource); + + hasKrb5 = true; + } + // register Flink Jar with remote HDFS LocalResource flinkJar = Records.newRecord(LocalResource.class); { @@ -563,6 +643,16 @@ public static ContainerLaunchContext createTaskManagerContext( taskManagerLocalResources.put("flink.jar", flinkJar); taskManagerLocalResources.put("flink-conf.yaml", flinkConf); + //To support Yarn Secure Integration Test Scenario + if(yarnConfResource != null && krb5ConfResource != null) { + taskManagerLocalResources.put(Utils.YARN_SITE_FILE_NAME, yarnConfResource); + taskManagerLocalResources.put(Utils.KRB5_FILE_NAME, krb5ConfResource); + } + + if(keytabResource != null) { + taskManagerLocalResources.put(Utils.KEYTAB_FILE_NAME, keytabResource); + } + // prepare additional files to be shipped for (String pathStr : shipListString.split(",")) { if (!pathStr.isEmpty()) { @@ -582,7 +672,7 @@ public static ContainerLaunchContext createTaskManagerContext( String launchCommand = BootstrapTools.getTaskManagerShellCommand( flinkConfig, tmParams, ".", ApplicationConstants.LOG_DIR_EXPANSION_VAR, - hasLogback, hasLog4j, taskManagerMainClass); + hasLogback, hasLog4j, hasKrb5, taskManagerMainClass); log.info("Starting TaskManagers with command: " + launchCommand); @@ -597,11 +687,17 @@ public static ContainerLaunchContext createTaskManagerContext( containerEnv.put(ENV_FLINK_CLASSPATH, classPathString); Utils.setupYarnClassPath(yarnConfig, containerEnv); - containerEnv.put(YarnConfigKeys.ENV_CLIENT_USERNAME, yarnClientUsername); + containerEnv.put(YarnConfigKeys.ENV_HADOOP_USER_NAME, UserGroupInformation.getCurrentUser().getUserName()); + + if(remoteKeytabPath != null && remoteKeytabPrincipal != null) { + containerEnv.put(YarnConfigKeys.KEYTAB_PATH, remoteKeytabPath); + containerEnv.put(YarnConfigKeys.KEYTAB_PRINCIPAL, remoteKeytabPrincipal); + } ctx.setEnvironment(containerEnv); try (DataOutputBuffer dob = new DataOutputBuffer()) { + LOG.debug("Adding security tokens to Task Manager Container launch Context...."); UserGroupInformation user = UserGroupInformation.getCurrentUser(); Credentials credentials = user.getCredentials(); credentials.writeTokenStorageToStream(dob); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnConfigKeys.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnConfigKeys.java index b14d7b7bdf18b..ada241caccf9e 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnConfigKeys.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnConfigKeys.java @@ -32,7 +32,6 @@ public class YarnConfigKeys { public final static String ENV_APP_ID = "_APP_ID"; public static final String ENV_CLIENT_HOME_DIR = "_CLIENT_HOME_DIR"; public static final String ENV_CLIENT_SHIP_FILES = "_CLIENT_SHIP_FILES"; - public static final String ENV_CLIENT_USERNAME = "_CLIENT_USERNAME"; public static final String ENV_SLOTS = "_SLOTS"; public static final String ENV_DETACHED = "_DETACHED"; public static final String ENV_DYNAMIC_PROPERTIES = "_DYNAMIC_PROPERTIES"; @@ -41,8 +40,13 @@ public class YarnConfigKeys { public final static String FLINK_JAR_PATH = "_FLINK_JAR_PATH"; // the Flink jar resource location (in HDFS). + public final static String KEYTAB_PATH = "_KEYTAB_PATH"; + public final static String KEYTAB_PRINCIPAL = "_KEYTAB_PRINCIPAL"; + public final static String ENV_HADOOP_USER_NAME = "HADOOP_USER_NAME"; public static final String ENV_ZOOKEEPER_NAMESPACE = "_ZOOKEEPER_NAMESPACE"; + public static final String ENV_KRB5_PATH = "_KRB5_PATH"; + public static final String ENV_YARN_SITE_XML_PATH = "_YARN_SITE_XML_PATH"; // ------------------------------------------------------------------------ diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskManagerRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskManagerRunner.java index 9638137c05085..c70a30bdace80 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskManagerRunner.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskManagerRunner.java @@ -18,22 +18,22 @@ package org.apache.flink.yarn; +import java.io.File; import java.io.IOException; -import java.security.PrivilegedAction; import java.util.Map; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.security.SecurityContext; import org.apache.flink.runtime.taskmanager.TaskManager; import org.apache.flink.runtime.util.EnvironmentInformation; import org.apache.flink.runtime.util.JvmShutdownSafeguard; import org.apache.flink.runtime.util.SignalHandler; import org.apache.flink.util.Preconditions; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.yarn.api.ApplicationConstants.Environment; import org.slf4j.Logger; @@ -64,8 +64,18 @@ public static void runYarnTaskManager(String[] args, final Class envs = System.getenv(); - final String yarnClientUsername = envs.get(YarnConfigKeys.ENV_CLIENT_USERNAME); + final String yarnClientUsername = envs.get(YarnConfigKeys.ENV_HADOOP_USER_NAME); final String localDirs = envs.get(Environment.LOCAL_DIRS.key()); + LOG.info("Current working/local Directory: {}", localDirs); + + final String currDir = envs.get(Environment.PWD.key()); + LOG.info("Current working Directory: {}", currDir); + + final String remoteKeytabPath = envs.get(YarnConfigKeys.KEYTAB_PATH); + LOG.info("TM: remoteKeytabPath obtained {}", remoteKeytabPath); + + final String remoteKeytabPrincipal = envs.get(YarnConfigKeys.KEYTAB_PRINCIPAL); + LOG.info("TM: remoteKeytabPrincipal obtained {}", remoteKeytabPrincipal); // configure local directory String flinkTempDirs = configuration.getString(ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, null); @@ -78,34 +88,66 @@ public static void runYarnTaskManager(String[] args, final Class toks : UserGroupInformation.getCurrentUser().getTokens()) { - ugi.addToken(toks); + String keytabPath = null; + if(remoteKeytabPath != null) { + File f = new File(currDir, Utils.KEYTAB_FILE_NAME); + keytabPath = f.getAbsolutePath(); + LOG.info("keytabPath: {}", keytabPath); } + UserGroupInformation currentUser = UserGroupInformation.getCurrentUser(); + + LOG.info("YARN daemon is running as: {} Yarn client user obtainer: {}", + currentUser.getShortUserName(), yarnClientUsername ); + // Infer the resource identifier from the environment variable String containerID = Preconditions.checkNotNull(envs.get(YarnFlinkResourceManager.ENV_FLINK_CONTAINER_ID)); final ResourceID resourceId = new ResourceID(containerID); LOG.info("ResourceID assigned for this container: {}", resourceId); - ugi.doAs(new PrivilegedAction() { - @Override - public Object run() { - try { - TaskManager.selectNetworkInterfaceAndRunTaskManager(configuration, resourceId, taskManager); - } - catch (Throwable t) { - LOG.error("Error while starting the TaskManager", t); - System.exit(TaskManager.STARTUP_FAILURE_RETURN_CODE()); - } - return null; + try { + + SecurityContext.SecurityConfiguration sc = new SecurityContext.SecurityConfiguration(); + + //To support Yarn Secure Integration Test Scenario + File krb5Conf = new File(currDir, Utils.KRB5_FILE_NAME); + if(krb5Conf.exists() && krb5Conf.canRead()) { + String krb5Path = krb5Conf.getAbsolutePath(); + LOG.info("KRB5 Conf: {}", krb5Path); + org.apache.hadoop.conf.Configuration conf = new org.apache.hadoop.conf.Configuration(); + conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos"); + conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION, "true"); + sc.setHadoopConfiguration(conf); + } + + if(keytabPath != null && remoteKeytabPrincipal != null) { + configuration.setString(ConfigConstants.SECURITY_KEYTAB_KEY, keytabPath); + configuration.setString(ConfigConstants.SECURITY_PRINCIPAL_KEY, remoteKeytabPrincipal); } - }); + configuration.setString(ConfigConstants.FLINK_BASE_DIR_PATH_KEY, currDir); + + SecurityContext.install(sc.setFlinkConfiguration(configuration)); + + SecurityContext.getInstalled().runSecured(new SecurityContext.FlinkSecuredRunner() { + @Override + public Integer run() { + try { + TaskManager.selectNetworkInterfaceAndRunTaskManager(configuration, resourceId, taskManager); + } + catch (Throwable t) { + LOG.error("Error while starting the TaskManager", t); + System.exit(TaskManager.STARTUP_FAILURE_RETURN_CODE()); + } + return null; + } + }); + } catch(Exception e) { + LOG.error("Exception occurred while launching Task Manager. Reason: {}", e); + throw new RuntimeException(e); + } + } -} +} \ No newline at end of file diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java index 8f02a1cef15a4..b5364f030b944 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java @@ -24,11 +24,14 @@ import org.apache.commons.cli.Options; import org.apache.commons.cli.PosixParser; import org.apache.commons.lang3.StringUtils; +import org.apache.flink.client.CliFrontend; import org.apache.flink.client.cli.CliFrontendParser; import org.apache.flink.client.cli.CustomCommandLine; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.runtime.security.SecurityContext; import org.apache.flink.yarn.AbstractYarnClusterDescriptor; import org.apache.flink.yarn.YarnClusterDescriptor; import org.apache.flink.yarn.YarnClusterClient; @@ -460,9 +463,27 @@ public static void runInteractiveCli(YarnClusterClient yarnCluster, boolean read } } - public static void main(String[] args) { - FlinkYarnSessionCli cli = new FlinkYarnSessionCli("", ""); // no prefix for the YARN session - System.exit(cli.run(args)); + public static void main(final String[] args) { + final FlinkYarnSessionCli cli = new FlinkYarnSessionCli("", "", true); // no prefix for the YARN session + + String confDirPath = CliFrontend.getConfigurationDirectoryFromEnv(); + GlobalConfiguration.loadConfiguration(confDirPath); + Configuration flinkConfiguration = GlobalConfiguration.loadConfiguration(); + flinkConfiguration.setString(ConfigConstants.FLINK_BASE_DIR_PATH_KEY, confDirPath); + try { + SecurityContext.install(new SecurityContext.SecurityConfiguration().setFlinkConfiguration(flinkConfiguration)); + int retCode = SecurityContext.getInstalled().runSecured(new SecurityContext.FlinkSecuredRunner() { + @Override + public Integer run() { + return cli.run(args); + } + }); + System.exit(retCode); + } catch(Exception e) { + e.printStackTrace(); + LOG.error("Exception Occured. Reason: {}", e); + return; + } } @Override @@ -523,6 +544,7 @@ public YarnClusterClient createCluster(String applicationName, CommandLine cmdLi try { return yarnClusterDescriptor.deploy(); } catch (Exception e) { + LOG.error("Error while deploying YARN cluster: "+e.getMessage(), e); throw new RuntimeException("Error deploying the YARN cluster", e); } diff --git a/pom.xml b/pom.xml index d92944fa87f83..102121fd64801 100644 --- a/pom.xml +++ b/pom.xml @@ -107,6 +107,13 @@ under the License. 2.7.4 3.1.0 4.11 + + 2.7.2 diff --git a/tools/log4j-travis.properties b/tools/log4j-travis.properties index 53379b4ed3084..476cee34b7f43 100644 --- a/tools/log4j-travis.properties +++ b/tools/log4j-travis.properties @@ -45,3 +45,6 @@ log4j.logger.org.apache.flink.runtime.leaderretrieval=DEBUG # the tests log4j.logger.org.apache.flink.yarn.YARNSessionFIFOITCase=INFO, console log4j.logger.org.apache.flink.yarn.YARNSessionCapacitySchedulerITCase=INFO, console +log4j.logger.org.apache.flink.streaming.connectors.kafka=INFO, console +log4j.logger.org.I0Itec.zkclient=INFO, console +log4j.logger.org.apache.flink.streaming.connectors.kafka.internals.SimpleConsumerThread=OFF \ No newline at end of file From 514bfe7e1a0a0f033c0be8e019f48b2b91439e2f Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Thu, 1 Sep 2016 12:49:53 +0200 Subject: [PATCH 061/299] [FLINK-3929] conditionally skip RollingSinkSecuredITCase - for now, we skip this test class until Hadoop version 3.x.x. --- .../fs/RollingSinkSecuredITCase.java | 37 +++++++++++++++---- .../connectors/kafka/KafkaTestBase.java | 8 +++- 2 files changed, 35 insertions(+), 10 deletions(-) diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java b/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java index 86cedaf307a7c..930ddd215b854 100644 --- a/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java +++ b/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -31,9 +31,10 @@ import org.apache.hadoop.http.HttpConfig; import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.util.VersionInfo; import org.junit.AfterClass; +import org.junit.Assume; import org.junit.BeforeClass; -import org.junit.Ignore; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -57,15 +58,31 @@ /** * Tests for running {@link RollingSinkSecuredITCase} which is an extension of {@link RollingSink} in secure environment + * Note: only executed for Hadoop version > 3.x.x */ - -//The test is disabled since MiniDFS secure run requires lower order ports to be used. -//We can enable the test when the fix is available (HDFS-9213) -@Ignore public class RollingSinkSecuredITCase extends RollingSinkITCase { protected static final Logger LOG = LoggerFactory.getLogger(RollingSinkSecuredITCase.class); + /** + * Skips all tests if the Hadoop version doesn't match. + * We can't run this test class until HDFS-9213 is fixed which allows a secure DataNode + * to bind to non-privileged ports for testing. + * For now, we skip this test class until Hadoop version 3.x.x. + */ + private static void skipIfHadoopVersionIsNotAppropriate() { + // Skips all tests if the Hadoop version doesn't match + String hadoopVersionString = VersionInfo.getVersion(); + String[] split = hadoopVersionString.split("\\."); + if (split.length != 3) { + throw new IllegalStateException("Hadoop version was not of format 'X.X.X': " + hadoopVersionString); + } + Assume.assumeTrue( + // check whether we're running Hadoop version >= 3.x.x + Integer.parseInt(split[0]) >= 3 + ); + } + /* * override super class static methods to avoid creating MiniDFS and MiniFlink with wrong configurations * and out-of-order sequence for secure cluster @@ -85,6 +102,8 @@ public static void destroyHDFS() {} @BeforeClass public static void startSecureCluster() throws Exception { + skipIfHadoopVersionIsNotAppropriate(); + LOG.info("starting secure cluster environment for testing"); dataDir = tempFolder.newFolder(); @@ -143,7 +162,9 @@ public static void teardownSecureCluster() throws Exception { TestStreamEnvironment.unsetAsContext(); stopCluster(cluster, TestBaseUtils.DEFAULT_TIMEOUT); - hdfsCluster.shutdown(); + if (hdfsCluster != null) { + hdfsCluster.shutdown(); + } SecureTestEnvironment.cleanup(); } @@ -229,4 +250,4 @@ public void testNonRollingAvroKeyValueWithCompressionWriter() throws Exception { @Override public void testDateTimeRollingStringWriter() throws Exception {} -} \ No newline at end of file +} diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java index afdd158e35eca..5cec4f0414cd7 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java @@ -110,7 +110,7 @@ public static void shutDownServices() { } protected static Configuration getFlinkConfiguration() { - Configuration flinkConfig = new Configuration();; + Configuration flinkConfig = new Configuration(); flinkConfig.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1); flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 8); flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 16); @@ -134,7 +134,11 @@ protected static void startClusters(boolean secureMode) throws ClassNotFoundExce brokerConnectionStrings = kafkaServer.getBrokerConnectionString(); - if(kafkaServer.isSecureRunSupported() && secureMode) { + if (secureMode) { + if (!kafkaServer.isSecureRunSupported()) { + throw new IllegalStateException( + "Attempting to test in secure mode but secure mode not supported by the KafkaTestEnvironment."); + } secureProps = kafkaServer.getSecureProperties(); } From a5634ca81179b783d9f49d65267482a0cb6ad475 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Tue, 20 Sep 2016 15:41:35 +0200 Subject: [PATCH 062/299] [FLINK-3929] additional fixes for keytab security - load flink-jaas.conf from classpath - avoid using undocumented flink base dir config entry - enable test cases to run on MacOS - unify suffix of secure test cases - fix error logging and reporting This closes #2275 --- .../org/apache/flink/client/CliFrontend.java | 3 - .../runtime/security/SecurityContext.java | 67 ++++++++----------- .../src/main/resources}/flink-jaas.conf | 2 +- .../runtime/taskmanager/TaskManager.scala | 2 - .../runtime/security/SecurityContextTest.java | 4 +- .../fs/RollingSinkSecuredITCase.java | 1 - ...Case.java => Kafka09SecuredRunITCase.java} | 8 +-- .../test/util/SecureTestEnvironment.java | 23 +++---- .../org/apache/flink/yarn/YarnTestBase.java | 3 +- .../yarn/YarnApplicationMasterRunner.java | 1 - .../flink/yarn/YarnTaskManagerRunner.java | 5 +- .../flink/yarn/cli/FlinkYarnSessionCli.java | 32 +++------ 12 files changed, 55 insertions(+), 96 deletions(-) rename {flink-dist/src/main/flink-bin/conf => flink-runtime/src/main/resources}/flink-jaas.conf (99%) rename flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/{Kafka09SecureRunITCase.java => Kafka09SecuredRunITCase.java} (92%) diff --git a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java index 575ffad79e46d..0711758752cb5 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java +++ b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java @@ -161,9 +161,6 @@ public CliFrontend(String configDir) throws Exception { "filesystem scheme from configuration.", e); } - this.config.setString(ConfigConstants.FLINK_BASE_DIR_PATH_KEY, configDirectory.getAbsolutePath() - + ".." + File.separator); - this.clientTimeout = AkkaUtils.getClientTimeout(config); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityContext.java b/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityContext.java index 4b8b69bc5f816..be6611fcf7c4a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityContext.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityContext.java @@ -22,7 +22,8 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.util.Preconditions; import org.apache.hadoop.io.Text; import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.UserGroupInformation; @@ -33,7 +34,12 @@ import javax.security.auth.Subject; import java.io.File; +import java.io.IOException; +import java.io.InputStream; import java.lang.reflect.Method; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardCopyOption; import java.security.PrivilegedExceptionAction; import java.util.Collection; @@ -170,15 +176,12 @@ public static void install(SecurityConfiguration config) throws Exception { * Kafka current code behavior. */ private static void populateSystemSecurityProperties(Configuration configuration) { + Preconditions.checkNotNull(configuration, "The supplied configuation was null"); //required to be empty for Kafka but we will override the property //with pseudo JAAS configuration file if SASL auth is enabled for ZK System.setProperty(JAVA_SECURITY_AUTH_LOGIN_CONFIG, ""); - if(configuration == null) { - return; - } - boolean disableSaslClient = configuration.getBoolean(ConfigConstants.ZOOKEEPER_SASL_DISABLE, ConfigConstants.DEFAULT_ZOOKEEPER_SASL_DISABLE); if(disableSaslClient) { @@ -188,46 +191,26 @@ private static void populateSystemSecurityProperties(Configuration configuration return; } - String baseDir = configuration.getString(ConfigConstants.FLINK_BASE_DIR_PATH_KEY, null); - if(baseDir == null) { - String message = "SASL auth is enabled for ZK but unable to locate pseudo Jaas config " + - "since " + ConfigConstants.FLINK_BASE_DIR_PATH_KEY + " is not provided"; - LOG.error(message); - throw new IllegalConfigurationException(message); - } - - File f = new File(baseDir); - if(!f.exists() || !f.isDirectory()) { - LOG.error("Invalid flink base directory {} configuration provided", baseDir); - throw new IllegalConfigurationException("Invalid flink base directory configuration provided"); - } - - File jaasConfigFile = new File(f, JAAS_CONF_FILENAME); - - if (!jaasConfigFile.exists() || !jaasConfigFile.isFile()) { - - //check if there is a conf directory - File confDir = new File(f, "conf"); - if(!confDir.exists() || !confDir.isDirectory()) { - LOG.error("Could not locate " + JAAS_CONF_FILENAME); - throw new IllegalConfigurationException("Could not locate " + JAAS_CONF_FILENAME); - } - - jaasConfigFile = new File(confDir, JAAS_CONF_FILENAME); - - if (!jaasConfigFile.exists() || !jaasConfigFile.isFile()) { - LOG.error("Could not locate " + JAAS_CONF_FILENAME); - throw new IllegalConfigurationException("Could not locate " + JAAS_CONF_FILENAME); - } + // load Jaas config file to initialize SASL + final File jaasConfFile; + try { + Path jaasConfPath = Files.createTempFile(JAAS_CONF_FILENAME, ""); + InputStream jaasConfStream = SecurityContext.class.getClassLoader().getResourceAsStream(JAAS_CONF_FILENAME); + Files.copy(jaasConfStream, jaasConfPath, StandardCopyOption.REPLACE_EXISTING); + jaasConfFile = jaasConfPath.toFile(); + jaasConfFile.deleteOnExit(); + } catch (IOException e) { + throw new RuntimeException("SASL auth is enabled for ZK but unable to " + + "locate pseudo Jaas config provided with Flink", e); } LOG.info("Enabling {} property with pseudo JAAS config file: {}", - JAVA_SECURITY_AUTH_LOGIN_CONFIG, jaasConfigFile); + JAVA_SECURITY_AUTH_LOGIN_CONFIG, jaasConfFile.getAbsolutePath()); //ZK client module lookup the configuration to handle SASL. //https://github.com/sgroschupf/zkclient/blob/master/src/main/java/org/I0Itec/zkclient/ZkClient.java#L900 - System.setProperty(JAVA_SECURITY_AUTH_LOGIN_CONFIG, jaasConfigFile.getAbsolutePath()); - System.setProperty(ZOOKEEPER_SASL_CLIENT,"true"); + System.setProperty(JAVA_SECURITY_AUTH_LOGIN_CONFIG, jaasConfFile.getAbsolutePath()); + System.setProperty(ZOOKEEPER_SASL_CLIENT, "true"); String zkSaslServiceName = configuration.getString(ConfigConstants.ZOOKEEPER_SASL_SERVICE_NAME, null); if(!StringUtils.isBlank(zkSaslServiceName)) { @@ -250,6 +233,10 @@ public static class SecurityConfiguration { String principal; + public SecurityConfiguration() { + this.flinkConf = GlobalConfiguration.loadConfiguration(); + } + public String getKeytab() { return keytab; } @@ -310,4 +297,4 @@ public interface FlinkSecuredRunner { T run() throws Exception; } -} \ No newline at end of file +} diff --git a/flink-dist/src/main/flink-bin/conf/flink-jaas.conf b/flink-runtime/src/main/resources/flink-jaas.conf similarity index 99% rename from flink-dist/src/main/flink-bin/conf/flink-jaas.conf rename to flink-runtime/src/main/resources/flink-jaas.conf index d476e24151b83..7f0f06ba0e4f3 100644 --- a/flink-dist/src/main/flink-bin/conf/flink-jaas.conf +++ b/flink-runtime/src/main/resources/flink-jaas.conf @@ -23,4 +23,4 @@ sample { useKeyTab=false useTicketCache=true; -}; \ No newline at end of file +} diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index 8534ee1b71565..9e2feb565c1ba 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -1583,8 +1583,6 @@ object TaskManager { } } - conf.setString(ConfigConstants.FLINK_BASE_DIR_PATH_KEY, cliConfig.getConfigDir() + "/..") - conf } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/security/SecurityContextTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/security/SecurityContextTest.java index 5f3d76a71bef9..3c48e8f2dcd6c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/security/SecurityContextTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/security/SecurityContextTest.java @@ -35,7 +35,7 @@ public void testCreateInsecureHadoopCtx() { SecurityContext.SecurityConfiguration sc = new SecurityContext.SecurityConfiguration(); try { SecurityContext.install(sc); - assertEquals(UserGroupInformation.getLoginUser().getUserName(),getOSUserName()); + assertEquals(UserGroupInformation.getLoginUser().getUserName(), getOSUserName()); } catch (Exception e) { fail(e.getMessage()); } @@ -59,7 +59,7 @@ private String getOSUserName() throws Exception { if( osName.contains( "windows" ) ){ className = "com.sun.security.auth.module.NTSystem"; } - else if( osName.contains( "linux" ) ){ + else if( osName.contains( "linux" ) || osName.contains( "mac" ) ){ className = "com.sun.security.auth.module.UnixSystem"; } else if( osName.contains( "solaris" ) || osName.contains( "sunos" ) ){ diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java b/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java index 930ddd215b854..051175a0ee9ca 100644 --- a/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java +++ b/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java @@ -227,7 +227,6 @@ private static void startSecureFlinkClusterWithRecoveryModeEnabled() { TestStreamEnvironment.setAsContext(cluster, DEFAULT_PARALLELISM); } catch (Exception e) { - LOG.error("Exception occured while creating MiniFlink cluster. Reason: {}", e); throw new RuntimeException(e); } } diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09SecureRunITCase.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09SecuredRunITCase.java similarity index 92% rename from flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09SecureRunITCase.java rename to flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09SecuredRunITCase.java index d12ec65af4cbc..e7485377ef69e 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09SecureRunITCase.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09SecuredRunITCase.java @@ -29,14 +29,14 @@ /* * Kafka Secure Connection (kerberos) IT test case */ -public class Kafka09SecureRunITCase extends KafkaConsumerTestBase { +public class Kafka09SecuredRunITCase extends KafkaConsumerTestBase { - protected static final Logger LOG = LoggerFactory.getLogger(Kafka09SecureRunITCase.class); + protected static final Logger LOG = LoggerFactory.getLogger(Kafka09SecuredRunITCase.class); @BeforeClass public static void prepare() throws IOException, ClassNotFoundException { LOG.info("-------------------------------------------------------------------------"); - LOG.info(" Starting Kafka09SecureRunITCase "); + LOG.info(" Starting Kafka09SecuredRunITCase "); LOG.info("-------------------------------------------------------------------------"); SecureTestEnvironment.prepare(tempFolder); @@ -59,4 +59,4 @@ public void testMultipleTopics() throws Exception { runProduceConsumeMultipleTopics(); } -} \ No newline at end of file +} diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/SecureTestEnvironment.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/SecureTestEnvironment.java index 00b19f19dce49..b5e622b9d3262 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/SecureTestEnvironment.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/SecureTestEnvironment.java @@ -20,6 +20,7 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.runtime.security.SecurityContext; import org.apache.hadoop.minikdc.MiniKdc; import org.junit.rules.TemporaryFolder; @@ -60,12 +61,10 @@ public class SecureTestEnvironment { private static String hadoopServicePrincipal = null; - private static File baseDirForSecureRun = null; - public static void prepare(TemporaryFolder tempFolder) { try { - baseDirForSecureRun = tempFolder.newFolder(); + File baseDirForSecureRun = tempFolder.newFolder(); LOG.info("Base Directory for Secure Environment: {}", baseDirForSecureRun); String hostName = "localhost"; @@ -113,19 +112,17 @@ public static void prepare(TemporaryFolder tempFolder) { //See Yarn test case module for reference createJaasConfig(baseDirForSecureRun); SecurityContext.SecurityConfiguration ctx = new SecurityContext.SecurityConfiguration(); - Configuration flinkConfig = new Configuration(); + Configuration flinkConfig = GlobalConfiguration.loadConfiguration(); flinkConfig.setString(ConfigConstants.SECURITY_KEYTAB_KEY, testKeytab); flinkConfig.setString(ConfigConstants.SECURITY_PRINCIPAL_KEY, testPrincipal); flinkConfig.setBoolean(ConfigConstants.ZOOKEEPER_SASL_DISABLE, false); - flinkConfig.setString(ConfigConstants.FLINK_BASE_DIR_PATH_KEY, baseDirForSecureRun.getAbsolutePath()); ctx.setFlinkConfiguration(flinkConfig); TestingSecurityContext.install(ctx, getClientSecurityConfigurationMap()); - populateSystemEnvVariables(); + populateJavaPropertyVariables(); } catch(Exception e) { - LOG.error("Exception occured while preparing secure environment. Reason: {}", e); - throw new RuntimeException(e); + throw new RuntimeException("Exception occured while preparing secure environment.", e); } } @@ -145,14 +142,12 @@ public static void cleanup() { testPrincipal = null; testZkServerPrincipal = null; hadoopServicePrincipal = null; - baseDirForSecureRun = null; } - private static void populateSystemEnvVariables() { + private static void populateJavaPropertyVariables() { if(LOG.isDebugEnabled()) { - System.setProperty("FLINK_JAAS_DEBUG", "true"); System.setProperty("sun.security.krb5.debug", "true"); } @@ -165,7 +160,6 @@ private static void populateSystemEnvVariables() { private static void resetSystemEnvVariables() { System.clearProperty("java.security.krb5.conf"); - System.clearProperty("FLINK_JAAS_DEBUG"); System.clearProperty("sun.security.krb5.debug"); System.clearProperty("zookeeper.authProvider.1"); @@ -227,7 +221,7 @@ public static String getHadoopServicePrincipal() { } /* - * Helper method to create a temporary JAAS configuration file to ger around the Kafka and ZK SASL + * Helper method to create a temporary JAAS configuration file to get around the Kafka and ZK SASL * implementation lookup java.security.auth.login.config */ private static void createJaasConfig(File baseDirForSecureRun) { @@ -241,8 +235,7 @@ private static void createJaasConfig(File baseDirForSecureRun) { out.println("useTicketCache=true;"); out.println("};"); } catch (IOException e) { - LOG.error("Exception occured while trying to create JAAS config. Reason: {}", e.getMessage()); - throw new RuntimeException(e); + throw new RuntimeException("Exception occured while trying to create JAAS config.", e); } } diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java index 605aa443fe785..afdd400809910 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java @@ -427,8 +427,7 @@ private static void start(Configuration conf, String principal, String keytab) { out.println(ConfigConstants.SECURITY_PRINCIPAL_KEY + ": " + principal); out.println(""); } catch (IOException e) { - LOG.error("Exception occured while trying to append the security configurations. Reason: {}", e.getMessage()); - throw new RuntimeException(e); + throw new RuntimeException("Exception occured while trying to append the security configurations.", e); } String configDir = tempConfPathForSecureRun.getAbsolutePath(); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java index efb658a1b6c3b..b27876b035fdd 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java @@ -176,7 +176,6 @@ protected int run(String[] args) { flinkConfig.setString(ConfigConstants.SECURITY_KEYTAB_KEY, keytabPath); flinkConfig.setString(ConfigConstants.SECURITY_PRINCIPAL_KEY, remoteKeytabPrincipal); } - flinkConfig.setString(ConfigConstants.FLINK_BASE_DIR_PATH_KEY, currDir); SecurityContext.install(sc.setFlinkConfiguration(flinkConfig)); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskManagerRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskManagerRunner.java index c70a30bdace80..21ed52eae33b9 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskManagerRunner.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnTaskManagerRunner.java @@ -127,7 +127,6 @@ public static void runYarnTaskManager(String[] args, final Class() { - @Override - public Integer run() { - return cli.run(args); - } - }); - System.exit(retCode); - } catch(Exception e) { - e.printStackTrace(); - LOG.error("Exception Occured. Reason: {}", e); - return; - } + SecurityContext.install(new SecurityContext.SecurityConfiguration().setFlinkConfiguration(flinkConfiguration)); + int retCode = SecurityContext.getInstalled().runSecured(new SecurityContext.FlinkSecuredRunner() { + @Override + public Integer run() { + return cli.run(args); + } + }); + System.exit(retCode); } @Override @@ -544,7 +533,6 @@ public YarnClusterClient createCluster(String applicationName, CommandLine cmdLi try { return yarnClusterDescriptor.deploy(); } catch (Exception e) { - LOG.error("Error while deploying YARN cluster: "+e.getMessage(), e); throw new RuntimeException("Error deploying the YARN cluster", e); } From 5910c4c6945d3ee221f7938981e9915b0ab73e38 Mon Sep 17 00:00:00 2001 From: twalthr Date: Thu, 4 Aug 2016 17:01:08 +0200 Subject: [PATCH 063/299] [FLINK-3042] [FLINK-3060] [types] Define a way to let types create their own TypeInformation This closes #2337. --- docs/dev/types_serialization.md | 42 ++ .../flink/api/common/typeinfo/TypeInfo.java | 44 ++ .../api/common/typeinfo/TypeInfoFactory.java | 51 ++ .../api/common/typeinfo/TypeInformation.java | 24 +- .../api/java/typeutils/TupleTypeInfo.java | 13 +- .../api/java/typeutils/TypeExtractor.java | 260 ++++++++-- .../java/typeutils/TypeInfoFactoryTest.java | 469 ++++++++++++++++++ .../api/scala/codegen/TypeAnalyzer.scala | 25 + .../api/scala/codegen/TypeDescriptors.scala | 8 + .../scala/codegen/TypeInformationGen.scala | 22 + .../scala/typeutils/CaseClassTypeInfo.scala | 6 +- .../api/scala/typeutils/EitherTypeInfo.scala | 3 +- .../scala/typeutils/EnumValueTypeInfo.scala | 2 +- .../api/scala/typeutils/OptionTypeInfo.scala | 2 +- .../scala/typeutils/TraversableTypeInfo.scala | 3 +- .../api/scala/typeutils/TryTypeInfo.scala | 2 +- .../scala/typeutils/TypeInfoFactoryTest.scala | 157 ++++++ 17 files changed, 1079 insertions(+), 54 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeInfo.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeInfoFactory.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/java/typeutils/TypeInfoFactoryTest.java create mode 100644 flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/TypeInfoFactoryTest.scala diff --git a/docs/dev/types_serialization.md b/docs/dev/types_serialization.md index 364aeb8d84ebf..8a32491c4137d 100644 --- a/docs/dev/types_serialization.md +++ b/docs/dev/types_serialization.md @@ -251,3 +251,45 @@ env.getConfig().addDefaultKryoSerializer(Class type, Class { + public T0 myfield0; + public T1 myfield1; +} +{% endhighlight %} + +The factory supplying custom type information: +{% highlight java %} +public class MyTupleTypeInfoFactory extends TypeInfoFactory { + + @Override + public TypeInformation createTypeInfo(Type t, Map> genericParameters) { + return new MyTupleTypeInfo(genericParameters.get("T0"), genericParameters.get("T1")); + } +} +{% endhighlight %} + +The method `createTypeInfo(Type, Map>)` creates type information for the type the factory is targeted for. +The parameters provide additional information about the type itself as well as the type's generic type parameters if available. + +If your type contains generic parameters that might need to be derived from the input type of a Flink function, make sure to also +implement `org.apache.flink.api.common.typeinfo.TypeInformation#getGenericParameters` for a bidirectional mapping of generic +parameters to type information. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeInfo.java new file mode 100644 index 0000000000000..ce468278edf7e --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeInfo.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.common.typeinfo; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; +import java.lang.reflect.Type; +import org.apache.flink.annotation.Public; +import org.apache.flink.api.java.typeutils.TypeExtractor; + +/** + * Annotation for specifying a corresponding {@link TypeInfoFactory} that can produce + * {@link TypeInformation} for the annotated type. In a hierarchy of types the closest annotation + * that defines a factory will be chosen while traversing upwards, however, a globally registered + * factory has highest precedence (see {@link TypeExtractor#registerFactory(Type, Class)}). + */ +@Documented +@Target(ElementType.TYPE) +@Retention(RetentionPolicy.RUNTIME) +@Public +public @interface TypeInfo { + + Class value(); + +} diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeInfoFactory.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeInfoFactory.java new file mode 100644 index 0000000000000..ea15f3aa07500 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeInfoFactory.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.common.typeinfo; + +import java.lang.reflect.Type; +import java.util.Map; +import org.apache.flink.annotation.Public; +import org.apache.flink.api.java.typeutils.TypeExtractor; + +/** + * Base class for implementing a type information factory. A type information factory allows for + * plugging-in user-defined {@link TypeInformation} into the Flink type system. The factory is + * called during the type extraction phase if the corresponding type has been annotated with + * {@link TypeInfo}. In a hierarchy of types the closest factory will be chosen while traversing + * upwards, however, a globally registered factory has highest precedence + * (see {@link TypeExtractor#registerFactory(Type, Class)}). + * + * @param type for which {@link TypeInformation} is created + */ +@Public +public abstract class TypeInfoFactory { + + /** + * Creates type information for the type the factory is targeted for. The parameters provide + * additional information about the type itself as well as the type's generic type parameters. + * + * @param t the exact type the type information is created for; might also be a subclass of <T> + * @param genericParameters mapping of the type's generic type parameters to type information + * extracted with Flink's type extraction facilities; null values + * indicate that type information could not be extracted for this parameter + * @return type information for the type the factory is targeted for + */ + public abstract TypeInformation createTypeInfo(Type t, Map> genericParameters); + +} diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeInformation.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeInformation.java index 95eed6be27224..154ceb1c42210 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeInformation.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeInformation.java @@ -18,15 +18,16 @@ package org.apache.flink.api.common.typeinfo; +import java.util.Map; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.annotation.Public; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TypeExtractor; import java.io.Serializable; import java.util.Collections; -import java.util.List; /** * TypeInformation is the core class of Flink's type system. Flink requires a type information @@ -122,14 +123,25 @@ public abstract class TypeInformation implements Serializable { public abstract Class getTypeClass(); /** - * Returns the generic parameters of this type. + * Optional method for giving Flink's type extraction system information about the mapping + * of a generic type parameter to the type information of a subtype. This information is necessary + * in cases where type information should be deduced from an input type. * - * @return The list of generic parameters. This list can be empty. + * For instance, a method for a {@link Tuple2} would look like this: + * + * Map m = new HashMap(); + * m.put("T0", this.getTypeAt(0)); + * m.put("T1", this.getTypeAt(1)); + * return m; + * + * + * @return map of inferred subtypes; it does not have to contain all generic parameters as key; + * values may be null if type could not be inferred */ @PublicEvolving - public List> getGenericParameters() { - // Return an empty list as the default implementation - return Collections.emptyList(); + public Map> getGenericParameters() { + // return an empty map as the default implementation + return Collections.emptyMap(); } /** diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfo.java index d525ffb6a6d95..e2cd789ae48c6 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfo.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfo.java @@ -22,6 +22,8 @@ import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; +import java.util.Map; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.annotation.Public; import org.apache.flink.api.common.ExecutionConfig; @@ -169,7 +171,16 @@ public TypeComparator createTypeComparator(ExecutionConfig config) { ); } } - + + @Override + public Map> getGenericParameters() { + Map> m = new HashMap<>(types.length); + for (int i = 0; i < types.length; i++) { + m.put("T" + i, types[i]); + } + return m; + } + // -------------------------------------------------------------------------------------------- @Override diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java index a722d7224a8d4..a0b09f57fd0c3 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java @@ -28,9 +28,12 @@ import java.lang.reflect.Type; import java.lang.reflect.TypeVariable; import java.util.ArrayList; +import java.util.Collections; import java.util.HashSet; +import java.util.HashMap; import java.util.List; +import java.util.Map; import org.apache.avro.specific.SpecificRecordBase; import org.apache.commons.lang3.ClassUtils; @@ -56,6 +59,8 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInfoFactory; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.CompositeType; import org.apache.flink.api.java.functions.KeySelector; @@ -63,7 +68,8 @@ import org.apache.flink.api.java.tuple.Tuple0; import org.apache.flink.types.Either; import org.apache.flink.types.Value; - +import org.apache.flink.util.InstantiationUtil; +import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -109,6 +115,34 @@ protected TypeExtractor() { // only create instances for special use cases } + // -------------------------------------------------------------------------------------------- + // TypeInfoFactory registry + // -------------------------------------------------------------------------------------------- + + private static Map> registeredTypeInfoFactories = new HashMap<>(); + + /** + * Registers a type information factory globally for a certain type. Every following type extraction + * operation will use the provided factory for this type. The factory will have highest precedence + * for this type. In a hierarchy of types the registered factory has higher precedence than annotations + * at the same level but lower precedence than factories defined down the hierarchy. + * + * @param t type for which a new factory is registered + * @param factory type information factory that will produce {@link TypeInformation} + */ + private static void registerFactory(Type t, Class factory) { + Preconditions.checkNotNull(t, "Type parameter must not be null."); + Preconditions.checkNotNull(factory, "Factory parameter must not be null."); + + if (!TypeInfoFactory.class.isAssignableFrom(factory)) { + throw new IllegalArgumentException("Class is not a TypeInfoFactory."); + } + if (registeredTypeInfoFactories.containsKey(t)) { + throw new InvalidTypesException("A TypeInfoFactory for type '" + t + "' is already registered."); + } + registeredTypeInfoFactories.put(t, factory); + } + // -------------------------------------------------------------------------------------------- // Function specific methods // -------------------------------------------------------------------------------------------- @@ -592,9 +626,14 @@ private TypeInformation privateCreateTypeInfo(Type returnTy @SuppressWarnings({ "unchecked", "rawtypes" }) private TypeInformation createTypeInfoWithTypeHierarchy(ArrayList typeHierarchy, Type t, TypeInformation in1Type, TypeInformation in2Type) { - + + // check if type information can be created using a type factory + final TypeInformation typeFromFactory = createTypeInfoFromFactory(t, typeHierarchy, in1Type, in2Type); + if (typeFromFactory != null) { + return typeFromFactory; + } // check if type is a subclass of tuple - if (isClassType(t) && Tuple.class.isAssignableFrom(typeToClass(t))) { + else if (isClassType(t) && Tuple.class.isAssignableFrom(typeToClass(t))) { Type curT = t; // do not allow usage of Tuple as type @@ -622,7 +661,7 @@ private TypeInformation createTypeInfoWithTypeHierarchy(Arr typeHierarchy.add(curT); // create the type information for the subtypes - TypeInformation[] subTypesInfo = createSubTypesInfo(t, (ParameterizedType) curT, typeHierarchy, in1Type, in2Type); + final TypeInformation[] subTypesInfo = createSubTypesInfo(t, (ParameterizedType) curT, typeHierarchy, in1Type, in2Type, false); // type needs to be treated a pojo due to additional fields if (subTypesInfo == null) { if (t instanceof ParameterizedType) { @@ -655,7 +694,7 @@ else if (isClassType(t) && Either.class.isAssignableFrom(typeToClass(t))) { typeHierarchy.add(curT); // create the type information for the subtypes - TypeInformation[] subTypesInfo = createSubTypesInfo(t, (ParameterizedType) curT, typeHierarchy, in1Type, in2Type); + final TypeInformation[] subTypesInfo = createSubTypesInfo(t, (ParameterizedType) curT, typeHierarchy, in1Type, in2Type, false); // type needs to be treated a pojo due to additional fields if (subTypesInfo == null) { if (t instanceof ParameterizedType) { @@ -807,12 +846,40 @@ private TypeInformation createTypeInfoFromInputs(TypeVariable r return null; } - + + @SuppressWarnings({"unchecked", "rawtypes"}) private TypeInformation createTypeInfoFromInput(TypeVariable returnTypeVar, ArrayList inputTypeHierarchy, Type inType, TypeInformation inTypeInfo) { TypeInformation info = null; - + + // use a factory to find corresponding type information to type variable + final ArrayList factoryHierarchy = new ArrayList<>(inputTypeHierarchy); + final TypeInfoFactory factory = getClosestFactory(factoryHierarchy, inType); + if (factory != null) { + // the type that defines the factory is last in factory hierarchy + final Type factoryDefiningType = factoryHierarchy.get(factoryHierarchy.size() - 1); + // defining type has generics, the factory need to be asked for a mapping of subtypes to type information + if (factoryDefiningType instanceof ParameterizedType) { + final Type[] typeParams = typeToClass(factoryDefiningType).getTypeParameters(); + final Type[] actualParams = ((ParameterizedType) factoryDefiningType).getActualTypeArguments(); + // go thru all elements and search for type variables + for (int i = 0; i < actualParams.length; i++) { + final Map> componentInfo = inTypeInfo.getGenericParameters(); + final String typeParamName = typeParams[i].toString(); + if (!componentInfo.containsKey(typeParamName) || componentInfo.get(typeParamName) == null) { + throw new InvalidTypesException("TypeInformation '" + inTypeInfo.getClass().getSimpleName() + + "' does not supply a mapping of TypeVariable '" + typeParamName + "' to corresponding TypeInformation. " + + "Input type inference can only produce a result with this information. " + + "Please implement method 'TypeInformation.getGenericParameters()' for this."); + } + info = createTypeInfoFromInput(returnTypeVar, factoryHierarchy, actualParams[i], componentInfo.get(typeParamName)); + if (info != null) { + break; + } + } + } + } // the input is a type variable - if (inType instanceof TypeVariable) { + else if (inType instanceof TypeVariable) { inType = materializeTypeVariable(inputTypeHierarchy, (TypeVariable) inType); info = findCorrespondingInfo(returnTypeVar, inType, inTypeInfo, inputTypeHierarchy); } @@ -873,28 +940,30 @@ else if (inTypeInfo instanceof PojoTypeInfo) { * @param typeHierarchy necessary for type inference * @param in1Type necessary for type inference * @param in2Type necessary for type inference + * @param lenient decides whether exceptions should be thrown if a subtype can not be determined * @return array containing TypeInformation of sub types or null if definingType contains * more subtypes (fields) that defined */ private TypeInformation[] createSubTypesInfo(Type originalType, ParameterizedType definingType, - ArrayList typeHierarchy, TypeInformation in1Type, TypeInformation in2Type) { + ArrayList typeHierarchy, TypeInformation in1Type, TypeInformation in2Type, boolean lenient) { Type[] subtypes = new Type[definingType.getActualTypeArguments().length]; // materialize possible type variables for (int i = 0; i < subtypes.length; i++) { + final Type actualTypeArg = definingType.getActualTypeArguments()[i]; // materialize immediate TypeVariables - if (definingType.getActualTypeArguments()[i] instanceof TypeVariable) { - subtypes[i] = materializeTypeVariable(typeHierarchy, (TypeVariable) definingType.getActualTypeArguments()[i]); + if (actualTypeArg instanceof TypeVariable) { + subtypes[i] = materializeTypeVariable(typeHierarchy, (TypeVariable) actualTypeArg); } // class or parameterized type else { - subtypes[i] = definingType.getActualTypeArguments()[i]; + subtypes[i] = actualTypeArg; } } TypeInformation[] subTypesInfo = new TypeInformation[subtypes.length]; for (int i = 0; i < subtypes.length; i++) { - ArrayList subTypeHierarchy = new ArrayList(typeHierarchy); + final ArrayList subTypeHierarchy = new ArrayList<>(typeHierarchy); subTypeHierarchy.add(subtypes[i]); // sub type could not be determined with materializing // try to derive the type info of the TypeVariable from the immediate base child input as a last attempt @@ -902,7 +971,7 @@ private TypeInformation[] createSubTypesInfo(Type originalType, Pa subTypesInfo[i] = createTypeInfoFromInputs((TypeVariable) subtypes[i], subTypeHierarchy, in1Type, in2Type); // variable could not be determined - if (subTypesInfo[i] == null) { + if (subTypesInfo[i] == null && !lenient) { throw new InvalidTypesException("Type of TypeVariable '" + ((TypeVariable) subtypes[i]).getName() + "' in '" + ((TypeVariable) subtypes[i]).getGenericDeclaration() + "' could not be determined. This is most likely a type erasure problem. " @@ -910,25 +979,75 @@ private TypeInformation[] createSubTypesInfo(Type originalType, Pa + "all variables in the return type can be deduced from the input type(s)."); } } else { - subTypesInfo[i] = createTypeInfoWithTypeHierarchy(subTypeHierarchy, subtypes[i], in1Type, in2Type); + // create the type information of the subtype or null/exception + try { + subTypesInfo[i] = createTypeInfoWithTypeHierarchy(subTypeHierarchy, subtypes[i], in1Type, in2Type); + } catch (InvalidTypesException e) { + if (lenient) { + subTypesInfo[i] = null; + } else { + throw e; + } + } } } - Class originalTypeAsClass = null; - if (isClassType(originalType)) { - originalTypeAsClass = typeToClass(originalType); + // check that number of fields matches the number of subtypes + if (!lenient) { + Class originalTypeAsClass = null; + if (isClassType(originalType)) { + originalTypeAsClass = typeToClass(originalType); + } + checkNotNull(originalTypeAsClass, "originalType has an unexpected type"); + // check if the class we assumed to conform to the defining type so far is actually a pojo because the + // original type contains additional fields. + // check for additional fields. + int fieldCount = countFieldsInClass(originalTypeAsClass); + if(fieldCount > subTypesInfo.length) { + return null; + } } - checkNotNull(originalTypeAsClass, "originalType has an unexpected type"); - // check if the class we assumed to conform to the defining type so far is actually a pojo because the - // original type contains additional fields. - // check for additional fields. - int fieldCount = countFieldsInClass(originalTypeAsClass); - if(fieldCount > subTypesInfo.length) { + + return subTypesInfo; + } + + /** + * Creates type information using a factory if for this type or super types. Returns null otherwise. + */ + @SuppressWarnings("unchecked") + private TypeInformation createTypeInfoFromFactory( + Type t, ArrayList typeHierarchy, TypeInformation in1Type, TypeInformation in2Type) { + + final ArrayList factoryHierarchy = new ArrayList<>(typeHierarchy); + final TypeInfoFactory factory = getClosestFactory(factoryHierarchy, t); + if (factory == null) { return null; } - return subTypesInfo; + final Type factoryDefiningType = factoryHierarchy.get(factoryHierarchy.size() - 1); + + // infer possible type parameters from input + final Map> genericParams; + if (factoryDefiningType instanceof ParameterizedType) { + genericParams = new HashMap<>(); + final ParameterizedType paramDefiningType = (ParameterizedType) factoryDefiningType; + final Type[] args = typeToClass(paramDefiningType).getTypeParameters(); + + final TypeInformation[] subtypeInfo = createSubTypesInfo(t, paramDefiningType, factoryHierarchy, in1Type, in2Type, true); + assert subtypeInfo != null; + for (int i = 0; i < subtypeInfo.length; i++) { + genericParams.put(args[i].toString(), subtypeInfo[i]); + } + } else { + genericParams = Collections.emptyMap(); + } + + final TypeInformation createdTypeInfo = (TypeInformation) factory.createTypeInfo(t, genericParams); + if (createdTypeInfo == null) { + throw new InvalidTypesException("TypeInfoFactory returned invalid TypeInformation 'null'"); + } + return createdTypeInfo; } - + // -------------------------------------------------------------------------------------------- // Extract type parameters // -------------------------------------------------------------------------------------------- @@ -1253,6 +1372,31 @@ private static void validateInputContainsMethod(Method m, TypeInformation typ // Utility methods // -------------------------------------------------------------------------------------------- + /** + * Returns the type information factory for a type using the factory registry or annotations. + */ + @Internal + public static TypeInfoFactory getTypeInfoFactory(Type t) { + final Class factoryClass; + if (registeredTypeInfoFactories.containsKey(t)) { + factoryClass = registeredTypeInfoFactories.get(t); + } + else { + if (!isClassType(t) || !typeToClass(t).isAnnotationPresent(TypeInfo.class)) { + return null; + } + final TypeInfo typeInfoAnnotation = typeToClass(t).getAnnotation(TypeInfo.class); + factoryClass = typeInfoAnnotation.value(); + // check for valid factory class + if (!TypeInfoFactory.class.isAssignableFrom(factoryClass)) { + throw new InvalidTypesException("TypeInfo annotation does not specify a valid TypeInfoFactory."); + } + } + + // instantiate + return (TypeInfoFactory) InstantiationUtil.instantiate(factoryClass); + } + /** * @return number of items with equal type or same raw type */ @@ -1265,27 +1409,46 @@ private static int countTypeInHierarchy(ArrayList typeHierarchy, Type type } return count; } - + /** - * @param curT : start type - * @return Type The immediate child of the top class + * Traverses the type hierarchy of a type up until a certain stop class is found. + * + * @param t type for which a hierarchy need to be created + * @return type of the immediate child of the stop class */ - private static Type getTypeHierarchy(ArrayList typeHierarchy, Type curT, Class stopAtClass) { - // skip first one - if (typeHierarchy.size() > 0 && typeHierarchy.get(0) == curT && isClassType(curT)) { - curT = typeToClass(curT).getGenericSuperclass(); + private static Type getTypeHierarchy(ArrayList typeHierarchy, Type t, Class stopAtClass) { + while (!(isClassType(t) && typeToClass(t).equals(stopAtClass))) { + typeHierarchy.add(t); + t = typeToClass(t).getGenericSuperclass(); + + if (t == null) { + break; + } } - while (!(isClassType(curT) && typeToClass(curT).equals(stopAtClass))) { - typeHierarchy.add(curT); - curT = typeToClass(curT).getGenericSuperclass(); + return t; + } - if (curT == null) { + /** + * Traverses the type hierarchy up until a type information factory can be found. + * + * @param typeHierarchy hierarchy to be filled while traversing up + * @param t type for which a factory needs to be found + * @return closest type information factory or null if there is no factory in the type hierarchy + */ + private static TypeInfoFactory getClosestFactory(ArrayList typeHierarchy, Type t) { + TypeInfoFactory factory = null; + while (factory == null && isClassType(t) && !(typeToClass(t).equals(Object.class))) { + typeHierarchy.add(t); + factory = getTypeInfoFactory(t); + t = typeToClass(t).getGenericSuperclass(); + + if (t == null) { break; } } - return curT; + return factory; } - + private int countFieldsInClass(Class clazz) { int fieldCount = 0; for(Field field : clazz.getFields()) { // get all fields @@ -1486,17 +1649,26 @@ private static Type materializeTypeVariable(ArrayList typeHierarchy, TypeV * @return TypeInformation that describes the passed Class */ public static TypeInformation getForClass(Class clazz) { - return new TypeExtractor().privateGetForClass(clazz, new ArrayList()); + final ArrayList typeHierarchy = new ArrayList<>(); + typeHierarchy.add(clazz); + return new TypeExtractor().privateGetForClass(clazz, typeHierarchy); } private TypeInformation privateGetForClass(Class clazz, ArrayList typeHierarchy) { return privateGetForClass(clazz, typeHierarchy, null, null, null); } + @SuppressWarnings({ "unchecked", "rawtypes" }) private TypeInformation privateGetForClass(Class clazz, ArrayList typeHierarchy, ParameterizedType parameterizedType, TypeInformation in1Type, TypeInformation in2Type) { checkNotNull(clazz); + // check if type information can be produced using a factory + final TypeInformation typeFromFactory = createTypeInfoFromFactory(clazz, typeHierarchy, in1Type, in2Type); + if (typeFromFactory != null) { + return typeFromFactory; + } + // Object is handled as generic type info if (clazz.equals(Object.class)) { return new GenericTypeInfo<>(clazz); @@ -1859,6 +2031,14 @@ public static TypeInformation getForObject(X value) { private TypeInformation privateGetForObject(X value) { checkNotNull(value); + // check if type information can be produced using a factory + final ArrayList typeHierarchy = new ArrayList<>(); + typeHierarchy.add(value.getClass()); + final TypeInformation typeFromFactory = createTypeInfoFromFactory(value.getClass(), typeHierarchy, null, null); + if (typeFromFactory != null) { + return typeFromFactory; + } + // check if we can extract the types from tuples, otherwise work with the class if (value instanceof Tuple) { Tuple t = (Tuple) value; diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/TypeInfoFactoryTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/TypeInfoFactoryTest.java new file mode 100644 index 0000000000000..f0558790037e8 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/TypeInfoFactoryTest.java @@ -0,0 +1,469 @@ +/* + * 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.typeutils; + +import java.lang.reflect.Type; +import java.util.HashMap; +import java.util.Map; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.InvalidTypesException; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.typeinfo.TypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInfoFactory; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple1; +import org.apache.flink.api.java.tuple.Tuple2; +import org.junit.Test; + +import static org.apache.flink.api.common.typeinfo.BasicTypeInfo.BOOLEAN_TYPE_INFO; +import static org.apache.flink.api.common.typeinfo.BasicTypeInfo.DOUBLE_TYPE_INFO; +import static org.apache.flink.api.common.typeinfo.BasicTypeInfo.FLOAT_TYPE_INFO; +import static org.apache.flink.api.common.typeinfo.BasicTypeInfo.INT_TYPE_INFO; +import static org.apache.flink.api.common.typeinfo.BasicTypeInfo.STRING_TYPE_INFO; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * Tests for extracting {@link org.apache.flink.api.common.typeinfo.TypeInformation} from types + * using a {@link org.apache.flink.api.common.typeinfo.TypeInfoFactory} + */ +public class TypeInfoFactoryTest { + + @Test + public void testSimpleType() { + TypeInformation ti = TypeExtractor.createTypeInfo(IntLike.class); + assertEquals(INT_TYPE_INFO, ti); + + ti = TypeExtractor.getForClass(IntLike.class); + assertEquals(INT_TYPE_INFO, ti); + + ti = TypeExtractor.getForObject(new IntLike()); + assertEquals(INT_TYPE_INFO, ti); + } + + @Test + public void testMyEitherGenericType() { + MapFunction> f = new MyEitherMapper<>(); + TypeInformation ti = TypeExtractor.getMapReturnTypes(f, BOOLEAN_TYPE_INFO); + assertTrue(ti instanceof EitherTypeInfo); + EitherTypeInfo eti = (EitherTypeInfo) ti; + assertEquals(BOOLEAN_TYPE_INFO, eti.getLeftType()); + assertEquals(STRING_TYPE_INFO, eti.getRightType()); + } + + @Test + public void testMyOptionGenericType() { + TypeInformation>> inTypeInfo = new MyOptionTypeInfo<>( + new TupleTypeInfo>(BOOLEAN_TYPE_INFO, STRING_TYPE_INFO)); + MapFunction>, MyOption>> f = new MyOptionMapper<>(); + TypeInformation ti = TypeExtractor.getMapReturnTypes(f, inTypeInfo); + assertTrue(ti instanceof MyOptionTypeInfo); + MyOptionTypeInfo oti = (MyOptionTypeInfo) ti; + assertTrue(oti.getInnerType() instanceof TupleTypeInfo); + TupleTypeInfo tti = (TupleTypeInfo) oti.getInnerType(); + assertEquals(BOOLEAN_TYPE_INFO, tti.getTypeAt(0)); + assertEquals(BOOLEAN_TYPE_INFO, tti.getTypeAt(1)); + } + + @Test + public void testMyTuple() { + TypeInformation>> inTypeInfo = new TupleTypeInfo<>( + new MyTupleTypeInfo(DOUBLE_TYPE_INFO, STRING_TYPE_INFO)); + MapFunction>, Tuple1>> f = new MyTupleMapperL2<>(); + TypeInformation ti = TypeExtractor.getMapReturnTypes(f, inTypeInfo); + assertTrue(ti instanceof TupleTypeInfo); + TupleTypeInfo tti = (TupleTypeInfo) ti; + assertTrue(tti.getTypeAt(0) instanceof MyTupleTypeInfo); + MyTupleTypeInfo mtti = (MyTupleTypeInfo) tti.getTypeAt(0); + assertEquals(BOOLEAN_TYPE_INFO, mtti.getField0()); + assertEquals(DOUBLE_TYPE_INFO, mtti.getField1()); + } + + @Test + public void testMyTupleHierarchy() { + TypeInformation ti = TypeExtractor.createTypeInfo(MyTuple2.class); + assertTrue(ti instanceof MyTupleTypeInfo); + MyTupleTypeInfo mtti = (MyTupleTypeInfo) ti; + assertEquals(STRING_TYPE_INFO, mtti.getField0()); + assertEquals(BOOLEAN_TYPE_INFO, mtti.getField1()); + } + + @Test + public void testMyTupleHierarchyWithInference() { + TypeInformation>>> inTypeInfo = new TupleTypeInfo<>(new MyTupleTypeInfo<>( + new TupleTypeInfo>(FLOAT_TYPE_INFO), BOOLEAN_TYPE_INFO)); + MapFunction>>, Tuple1>>> f = new MyTuple3Mapper<>(); + TypeInformation ti = TypeExtractor.getMapReturnTypes(f, inTypeInfo); + assertTrue(ti instanceof TupleTypeInfo); + TupleTypeInfo tti = (TupleTypeInfo) ti; + assertTrue(tti.getTypeAt(0) instanceof MyTupleTypeInfo); + MyTupleTypeInfo mtti = (MyTupleTypeInfo) tti.getTypeAt(0); + assertEquals(new TupleTypeInfo<>(FLOAT_TYPE_INFO, STRING_TYPE_INFO), mtti.getField0()); + assertEquals(BOOLEAN_TYPE_INFO, mtti.getField1()); + } + + @Test(expected = InvalidTypesException.class) + public void testMissingTypeInfo() { + MapFunction f = new MyFaultyMapper(); + TypeExtractor.getMapReturnTypes(f, INT_TYPE_INFO); + } + + @Test(expected = InvalidTypesException.class) + public void testMissingTypeInference() { + MapFunction f = new MyFaultyMapper2(); + TypeExtractor.getMapReturnTypes(f, new MyFaultyTypeInfo()); + } + + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- + + public static class MyTuple3Mapper implements MapFunction>>, Tuple1>>> { + @Override + public Tuple1>> map(Tuple1>> value) throws Exception { + return null; + } + } + + public static class MyTuple3 extends MyTuple { + // empty + } + + public static class MyTuple2 extends MyTuple { + // empty + } + + public static class MyFaultyMapper2 implements MapFunction, MyFaulty> { + @Override + public MyFaulty map(MyFaulty value) throws Exception { + return null; + } + } + + public static class MyFaultyMapper implements MapFunction> { + @Override + public MyFaulty map(T value) throws Exception { + return null; + } + } + + @TypeInfo(FaultyTypeInfoFactory.class) + public static class MyFaulty { + // empty + } + + public static class FaultyTypeInfoFactory extends TypeInfoFactory { + @Override + public TypeInformation createTypeInfo(Type t, Map genericParameters) { + return null; + } + } + + public static class MyFaultyTypeInfo extends TypeInformation { + @Override + public boolean isBasicType() { + return false; + } + + @Override + public boolean isTupleType() { + return false; + } + + @Override + public int getArity() { + return 0; + } + + @Override + public int getTotalFields() { + return 0; + } + + @Override + public Class getTypeClass() { + return null; + } + + @Override + public boolean isKeyType() { + return false; + } + + @Override + public TypeSerializer createSerializer(ExecutionConfig config) { + return null; + } + + @Override + public String toString() { + return null; + } + + @Override + public boolean equals(Object obj) { + return false; + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public boolean canEqual(Object obj) { + return false; + } + } + + public static class MyTupleMapperL1 implements MapFunction>, Tuple1>> { + @Override + public Tuple1> map(Tuple1> value) throws Exception { + return null; + } + } + + public static class MyTupleMapperL2 extends MyTupleMapperL1 { + // empty + } + + @TypeInfo(MyTupleTypeInfoFactory.class) + public static class MyTuple { + // empty + } + + public static class MyTupleTypeInfoFactory extends TypeInfoFactory { + @Override + @SuppressWarnings("unchecked") + public TypeInformation createTypeInfo(Type t, Map> genericParameters) { + return new MyTupleTypeInfo(genericParameters.get("T0"), genericParameters.get("T1")); + } + } + + public static class MyTupleTypeInfo extends TypeInformation> { + private TypeInformation field0; + private TypeInformation field1; + + public TypeInformation getField0() { + return field0; + } + + public TypeInformation getField1() { + return field1; + } + + public MyTupleTypeInfo(TypeInformation field0, TypeInformation field1) { + this.field0 = field0; + this.field1 = field1; + } + + @Override + public boolean isBasicType() { + return false; + } + + @Override + public boolean isTupleType() { + return false; + } + + @Override + public int getArity() { + return 0; + } + + @Override + public int getTotalFields() { + return 0; + } + + @Override + public Class> getTypeClass() { + return null; + } + + @Override + public boolean isKeyType() { + return false; + } + + @Override + public TypeSerializer> createSerializer(ExecutionConfig config) { + return null; + } + + @Override + public String toString() { + return null; + } + + @Override + public boolean equals(Object obj) { + return false; + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public boolean canEqual(Object obj) { + return false; + } + + @Override + public Map> getGenericParameters() { + Map> map = new HashMap<>(2); + map.put("T0", field0); + map.put("T1", field1); + return map; + } + } + + public static class MyOptionMapper implements MapFunction>, MyOption>> { + @Override + public MyOption> map(MyOption> value) throws Exception { + return null; + } + } + + @TypeInfo(MyOptionTypeInfoFactory.class) + public static class MyOption { + // empty + } + + public static class MyOptionTypeInfoFactory extends TypeInfoFactory> { + @Override + @SuppressWarnings("unchecked") + public TypeInformation> createTypeInfo(Type t, Map> genericParams) { + return new MyOptionTypeInfo(genericParams.get("T")); + } + } + + public static class MyOptionTypeInfo extends TypeInformation> { + + private final TypeInformation innerType; + + public MyOptionTypeInfo(TypeInformation innerType) { + this.innerType = innerType; + } + + public TypeInformation getInnerType() { + return innerType; + } + + @Override + public boolean isBasicType() { + return false; + } + + @Override + public boolean isTupleType() { + return false; + } + + @Override + public int getArity() { + return 0; + } + + @Override + public int getTotalFields() { + return 0; + } + + @Override + public Class> getTypeClass() { + return null; + } + + @Override + public boolean isKeyType() { + return false; + } + + @Override + public TypeSerializer> createSerializer(ExecutionConfig config) { + return null; + } + + @Override + public String toString() { + return null; + } + + @Override + public boolean equals(Object obj) { + return false; + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public boolean canEqual(Object obj) { + return false; + } + + @Override + public Map> getGenericParameters() { + Map> map = new HashMap<>(1); + map.put("T", innerType); + return map; + } + } + + public static class MyEitherMapper implements MapFunction> { + @Override + public MyEither map(T value) throws Exception { + return null; + } + } + + @TypeInfo(MyEitherTypeInfoFactory.class) + public static class MyEither { + // empty + } + + public static class MyEitherTypeInfoFactory extends TypeInfoFactory> { + @Override + @SuppressWarnings("unchecked") + public TypeInformation> createTypeInfo(Type t, Map> genericParams) { + return new EitherTypeInfo(genericParams.get("A"), genericParams.get("B")); + } + } + + @TypeInfo(IntLikeTypeInfoFactory.class) + public static class IntLike { + // empty + } + + public static class IntLikeTypeInfoFactory extends TypeInfoFactory { + @Override + @SuppressWarnings("unchecked") + public TypeInformation createTypeInfo(Type t, Map> genericParams) { + return (TypeInformation) INT_TYPE_INFO; + } + } + +} diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeAnalyzer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeAnalyzer.scala index a8587ef5c7d24..11d5ec7dbbb63 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeAnalyzer.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeAnalyzer.scala @@ -51,6 +51,9 @@ private[flink] trait TypeAnalyzer[C <: Context] { this: MacroContextHolder[C] case TypeParameter() => TypeParameterDescriptor(id, tpe) + // type or super type defines type information factory + case FactoryType(baseType) => analyzeFactoryType(id, tpe, baseType) + case PrimitiveType(default, wrapper) => PrimitiveDescriptor(id, tpe, default, wrapper) case BoxedPrimitiveType(default, wrapper, box, unbox) => @@ -91,6 +94,19 @@ private[flink] trait TypeAnalyzer[C <: Context] { this: MacroContextHolder[C] } } + private def analyzeFactoryType( + id: Int, + tpe: Type, + baseType: Type): UDTDescriptor = { + val params: Seq[UDTDescriptor] = baseType match { + case TypeRef(_, _, args) => + args.map(analyze) + case _ => + Seq[UDTDescriptor]() + } + FactoryTypeDescriptor(id, tpe, baseType, params) + } + private def analyzeArray( id: Int, tpe: Type, @@ -438,6 +454,15 @@ private[flink] trait TypeAnalyzer[C <: Context] { this: MacroContextHolder[C] def unapply(tpe: Type): Boolean = tpe <:< typeOf[org.apache.flink.api.java.tuple.Tuple] } + private object FactoryType { + def unapply(tpe: Type): Option[Type] = { + val definingType = tpe.typeSymbol.asClass.baseClasses find { + _.annotations.exists(_.tpe =:= typeOf[org.apache.flink.api.common.typeinfo.TypeInfo]) + } + definingType.map(tpe.baseType) + } + } + private class UDTAnalyzerCache { private val caches = new DynamicVariable[Map[Type, RecursiveDescriptor]](Map()) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeDescriptors.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeDescriptors.scala index 4efa546ba1e09..9efde0f8ede0c 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeDescriptors.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeDescriptors.scala @@ -19,6 +19,7 @@ package org.apache.flink.api.scala.codegen import org.apache.flink.annotation.Internal +import scala.collection.Map import scala.language.postfixOps import scala.reflect.macros.Context @@ -53,6 +54,13 @@ private[flink] trait TypeDescriptors[C <: Context] { this: MacroContextHolder[C] case class TryDescriptor(id: Int, tpe: Type, elem: UDTDescriptor) extends UDTDescriptor + case class FactoryTypeDescriptor( + id: Int, + tpe: Type, + baseType: Type, + params: Seq[UDTDescriptor]) + extends UDTDescriptor + case class OptionDescriptor(id: Int, tpe: Type, elem: UDTDescriptor) extends UDTDescriptor case class BoxedPrimitiveDescriptor( diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala index ee0d1674b9025..9736e81f6c69a 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala @@ -52,6 +52,9 @@ private[flink] trait TypeInformationGen[C <: Context] { // We have this for internal use so that we can use it to recursively generate a tree of // TypeInformation from a tree of UDTDescriptor def mkTypeInfo[T: c.WeakTypeTag](desc: UDTDescriptor): c.Expr[TypeInformation[T]] = desc match { + + case f: FactoryTypeDescriptor => mkTypeInfoFromFactory(f) + case cc@CaseClassDescriptor(_, tpe, _, _, _) => mkCaseClassTypeInfo(cc)(c.WeakTypeTag(tpe).asInstanceOf[c.WeakTypeTag[Product]]) .asInstanceOf[c.Expr[TypeInformation[T]]] @@ -93,6 +96,25 @@ private[flink] trait TypeInformationGen[C <: Context] { case d => mkGenericTypeInfo(d) } + def mkTypeInfoFromFactory[T: c.WeakTypeTag](desc: FactoryTypeDescriptor) + : c.Expr[TypeInformation[T]] = { + + val tpeClazz = c.Expr[Class[T]](Literal(Constant(desc.tpe))) + val baseClazz = c.Expr[Class[T]](Literal(Constant(desc.baseType))) + + val typeInfos = desc.params map { p => mkTypeInfo(p)(c.WeakTypeTag(p.tpe)).tree } + val typeInfosList = c.Expr[List[TypeInformation[_]]](mkList(typeInfos.toList)) + + reify { + val factory = TypeExtractor.getTypeInfoFactory[T](baseClazz.splice) + val genericParameters = typeInfosList.splice + .zip(baseClazz.splice.getTypeParameters).map { case (typeInfo, typeParam) => + typeParam.getName -> typeInfo + }.toMap[String, TypeInformation[_]] + factory.createTypeInfo(tpeClazz.splice, genericParameters.asJava) + } + } + def mkCaseClassTypeInfo[T <: Product : c.WeakTypeTag]( desc: CaseClassDescriptor): c.Expr[TypeInformation[T]] = { val tpeClazz = c.Expr[Class[T]](Literal(Constant(desc.tpe))) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassTypeInfo.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassTypeInfo.scala index d658fdec3ce0d..2aecd7a68f358 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassTypeInfo.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassTypeInfo.scala @@ -47,8 +47,10 @@ abstract class CaseClassTypeInfo[T <: Product]( extends TupleTypeInfoBase[T](clazz, fieldTypes: _*) { @PublicEvolving - override def getGenericParameters: java.util.List[TypeInformation[_]] = { - typeParamTypeInfos.toList.asJava + override def getGenericParameters: java.util.Map[String, TypeInformation[_]] = { + typeParamTypeInfos.zipWithIndex.map { case (info, index) => + "T" + (index + 1) -> info + }.toMap[String, TypeInformation[_]].asJava } private val REGEX_INT_FIELD: String = "[0-9]+" diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EitherTypeInfo.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EitherTypeInfo.scala index 406f0735115b9..e89730951be0c 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EitherTypeInfo.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EitherTypeInfo.scala @@ -47,7 +47,8 @@ class EitherTypeInfo[A, B, T <: Either[A, B]]( @PublicEvolving override def getTypeClass = clazz @PublicEvolving - override def getGenericParameters = List[TypeInformation[_]](leftTypeInfo, rightTypeInfo).asJava + override def getGenericParameters = + Map[String, TypeInformation[_]]("A" -> leftTypeInfo, "B" -> rightTypeInfo).asJava @PublicEvolving def createSerializer(executionConfig: ExecutionConfig): TypeSerializer[T] = { diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueTypeInfo.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueTypeInfo.scala index 92d2704b5500d..efc6427dec7d9 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueTypeInfo.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueTypeInfo.scala @@ -46,7 +46,7 @@ class EnumValueTypeInfo[E <: Enumeration](val enum: E, val clazz: Class[E#Value] @PublicEvolving override def getTypeClass = clazz @PublicEvolving - override def getGenericParameters = List.empty[TypeInformation[_]].asJava + override def getGenericParameters = Map.empty[String, TypeInformation[_]].asJava @PublicEvolving diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/OptionTypeInfo.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/OptionTypeInfo.scala index 58ae77c4f0f3b..73fe5807524e4 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/OptionTypeInfo.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/OptionTypeInfo.scala @@ -44,7 +44,7 @@ class OptionTypeInfo[A, T <: Option[A]](private val elemTypeInfo: TypeInformatio @PublicEvolving override def getTypeClass = classOf[Option[_]].asInstanceOf[Class[T]] @PublicEvolving - override def getGenericParameters = List[TypeInformation[_]](elemTypeInfo).asJava + override def getGenericParameters = Map[String, TypeInformation[_]]("A" -> elemTypeInfo).asJava @PublicEvolving override def createComparator(ascending: Boolean, executionConfig: ExecutionConfig) = { diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableTypeInfo.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableTypeInfo.scala index 82fd8ae5d80c7..47fb0398e9020 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableTypeInfo.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableTypeInfo.scala @@ -46,7 +46,8 @@ abstract class TraversableTypeInfo[T <: TraversableOnce[E], E]( @PublicEvolving override def getTypeClass: Class[T] = clazz @PublicEvolving - override def getGenericParameters = List[TypeInformation[_]](elementTypeInfo).asJava + override def getGenericParameters = + Map[String, TypeInformation[_]]("A" -> elementTypeInfo).asJava @PublicEvolving def createSerializer(executionConfig: ExecutionConfig): TypeSerializer[T] diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TryTypeInfo.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TryTypeInfo.scala index 0a5a06da87caa..b09c353dfab63 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TryTypeInfo.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TryTypeInfo.scala @@ -46,7 +46,7 @@ class TryTypeInfo[A, T <: Try[A]](val elemTypeInfo: TypeInformation[A]) @PublicEvolving override def getTypeClass = classOf[Try[_]].asInstanceOf[Class[T]] @PublicEvolving - override def getGenericParameters = List[TypeInformation[_]](elemTypeInfo).asJava + override def getGenericParameters = Map[String, TypeInformation[_]]("T" -> elemTypeInfo).asJava @PublicEvolving def createSerializer(executionConfig: ExecutionConfig): TypeSerializer[T] = { diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/TypeInfoFactoryTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/TypeInfoFactoryTest.scala new file mode 100644 index 0000000000000..5873630698dc5 --- /dev/null +++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/TypeInfoFactoryTest.scala @@ -0,0 +1,157 @@ +/* + * 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.scala.typeutils + +import java.lang.reflect.Type +import java.util + +import org.apache.flink.api.common.ExecutionConfig +import org.apache.flink.api.common.typeinfo.BasicTypeInfo._ +import org.apache.flink.api.common.typeinfo.{TypeInfo, TypeInfoFactory, TypeInformation} +import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.java.typeutils.TypeInfoFactoryTest._ +import org.apache.flink.api.java.typeutils.{EitherTypeInfo => JavaEitherTypeInfo} +import org.apache.flink.api.scala._ +import org.apache.flink.api.scala.typeutils.TypeInfoFactoryTest._ +import org.apache.flink.util.TestLogger +import org.junit.Assert._ +import org.junit.Test +import org.scalatest.junit.JUnitSuiteLike + +class TypeInfoFactoryTest extends TestLogger with JUnitSuiteLike { + + @Test + def testSimpleType(): Unit = { + val ti = createTypeInformation[ScalaIntLike] + assertEquals(INT_TYPE_INFO, ti) + } + + @Test + def testMyTuple(): Unit = { + val ti = createTypeInformation[MyTuple[Double, String]] + assertTrue(ti.isInstanceOf[MyTupleTypeInfo[_, _]]) + val mtti = ti.asInstanceOf[MyTupleTypeInfo[_, _]] + assertEquals(DOUBLE_TYPE_INFO, mtti.getField0) + assertEquals(STRING_TYPE_INFO, mtti.getField1) + } + + @Test + def testMyTupleHierarchy() { + val ti = createTypeInformation[MyTuple2] + assertTrue(ti.isInstanceOf[MyTupleTypeInfo[_, _]]) + val mtti = ti.asInstanceOf[MyTupleTypeInfo[_, _]] + assertEquals(STRING_TYPE_INFO, mtti.getField0) + assertEquals(BOOLEAN_TYPE_INFO, mtti.getField1) + + val ti2 = createTypeInformation[MyScalaTupleClass] + assertTrue(ti2.isInstanceOf[MyTupleTypeInfo[_, _]]) + val mtti2 = ti2.asInstanceOf[MyTupleTypeInfo[_, _]] + assertEquals(STRING_TYPE_INFO, mtti.getField0) + assertEquals(BOOLEAN_TYPE_INFO, mtti.getField1) + } + + @Test + def testMyTupleHierarchyWithCaseClass(): Unit = { + val ti = createTypeInformation[MyScalaTupleCaseClass] + assertTrue(ti.isInstanceOf[MyTupleTypeInfo[_, _]]) + val mtti = ti.asInstanceOf[MyTupleTypeInfo[_, _]] + assertEquals(DOUBLE_TYPE_INFO, mtti.getField0) + assertEquals(BOOLEAN_TYPE_INFO, mtti.getField1) + } + + @Test + def testMyEitherGenericType(): Unit = { + val ti = createTypeInformation[MyScalaEither[String, (Double, Int)]] + assertTrue(ti.isInstanceOf[JavaEitherTypeInfo[_, _]]) + val eti = ti.asInstanceOf[JavaEitherTypeInfo[_, _]] + assertEquals(STRING_TYPE_INFO, eti.getLeftType) + assertTrue(eti.getRightType.isInstanceOf[CaseClassTypeInfo[_]]) + val cti = eti.getRightType.asInstanceOf[CaseClassTypeInfo[_]] + assertEquals(DOUBLE_TYPE_INFO, cti.getTypeAt(0)) + assertEquals(INT_TYPE_INFO, cti.getTypeAt(1)) + } + + @Test + def testScalaFactory(): Unit = { + val ti = createTypeInformation[MyScalaOption[Double]] + assertTrue(ti.isInstanceOf[MyScalaOptionTypeInfo]) + val moti = ti.asInstanceOf[MyScalaOptionTypeInfo] + assertEquals(DOUBLE_TYPE_INFO, moti.elementType) + } +} + +// -------------------------------------------------------------------------------------------- +// Utilities +// -------------------------------------------------------------------------------------------- + +object TypeInfoFactoryTest { + + @TypeInfo(classOf[IntLikeTypeInfoFactory]) + case class ScalaIntLike(myint: Int) + + class MyScalaTupleClass extends MyTuple2 + + case class MyScalaTupleCaseClass(additional: Boolean) extends MyTuple3[Double] + + @TypeInfo(classOf[MyEitherTypeInfoFactory[_, _]]) + class MyScalaEither[A, B] { + // do nothing here + } + + @TypeInfo(classOf[MyScalaOptionTypeInfoFactory]) + class MyScalaOption[Z] { + // do nothing here + } + + class MyScalaOptionTypeInfoFactory extends TypeInfoFactory[MyOption[_]] { + + override def createTypeInfo( + t: Type, + genericParameters: util.Map[String, TypeInformation[_]]) + : TypeInformation[MyOption[_]] = { + new MyScalaOptionTypeInfo(genericParameters.get("Z")) + } + } + + class MyScalaOptionTypeInfo(val elementType: TypeInformation[_]) + extends TypeInformation[MyOption[_]] { + + override def isBasicType: Boolean = ??? + + override def isTupleType: Boolean = ??? + + override def getArity: Int = ??? + + override def getTotalFields: Int = ??? + + override def getTypeClass: Class[MyOption[_]] = ??? + + override def isKeyType: Boolean = ??? + + override def createSerializer(config: ExecutionConfig): TypeSerializer[MyOption[_]] = ??? + + override def canEqual(obj: scala.Any): Boolean = ??? + + override def hashCode(): Int = ??? + + override def toString: String = ??? + + override def equals(obj: scala.Any): Boolean = ??? + } +} From a723c63e5b76e3aa8f10565cbee23813d89a86d7 Mon Sep 17 00:00:00 2001 From: David Anderson Date: Tue, 20 Sep 2016 14:54:44 +0200 Subject: [PATCH 064/299] [FLINK-4654] [docs] Small improvements to the docs. This closes #2525 --- docs/dev/datastream_api.md | 6 +++--- docs/dev/libs/cep.md | 9 +++++---- docs/dev/state.md | 17 ++++++++--------- docs/dev/state_backends.md | 6 +++--- docs/dev/windows.md | 4 ++-- docs/quickstart/run_example_quickstart.md | 23 ++++++++++++----------- 6 files changed, 33 insertions(+), 32 deletions(-) diff --git a/docs/dev/datastream_api.md b/docs/dev/datastream_api.md index 2dd7842c570d8..425dd6acc8f19 100644 --- a/docs/dev/datastream_api.md +++ b/docs/dev/datastream_api.md @@ -357,7 +357,7 @@ windowedStream.reduce (new ReduceFunction() { The example function, when applied on the sequence (1,2,3,4,5), folds the sequence into the string "start-1-2-3-4-5":

{% highlight java %} -windowedStream.fold("start-", new FoldFunction() { +windowedStream.fold("start", new FoldFunction() { public String fold(String current, Integer value) { return current + "-" + value; } @@ -1324,7 +1324,7 @@ File-based: *IMPORTANT NOTES:* - 1. If the `watchType` is set to `FileProcessingMode.PROCESS_CONTINUOUSLY`, when a file is modified, its contents are re-processed entirely. This can brake the "exactly-once" semantics, as appending data at the end of a file will lead to **all** its contents being re-processed. + 1. If the `watchType` is set to `FileProcessingMode.PROCESS_CONTINUOUSLY`, when a file is modified, its contents are re-processed entirely. This can break the "exactly-once" semantics, as appending data at the end of a file will lead to **all** its contents being re-processed. 2. If the `watchType` is set to `FileProcessingMode.PROCESS_ONCE`, the source scans the path **once** and exits, without waiting for the readers to finish reading the file contents. Of course the readers will continue reading until all file contents are read. Closing the source leads to no more checkpoints after that point. This may lead to slower recovery after a node failure, as the job will resume reading from the last checkpoint. @@ -1382,7 +1382,7 @@ File-based: *IMPORTANT NOTES:* - 1. If the `watchType` is set to `FileProcessingMode.PROCESS_CONTINUOUSLY`, when a file is modified, its contents are re-processed entirely. This can brake the "exactly-once" semantics, as appending data at the end of a file will lead to **all** its contents being re-processed. + 1. If the `watchType` is set to `FileProcessingMode.PROCESS_CONTINUOUSLY`, when a file is modified, its contents are re-processed entirely. This can break the "exactly-once" semantics, as appending data at the end of a file will lead to **all** its contents being re-processed. 2. If the `watchType` is set to `FileProcessingMode.PROCESS_ONCE`, the source scans the path **once** and exits, without waiting for the readers to finish reading the file contents. Of course the readers will continue reading until all file contents are read. Closing the source leads to no more checkpoints after that point. This may lead to slower recovery after a node failure, as the job will resume reading from the last checkpoint. diff --git a/docs/dev/libs/cep.md b/docs/dev/libs/cep.md index 77266bc54898f..d27cf9f94b8ab 100644 --- a/docs/dev/libs/cep.md +++ b/docs/dev/libs/cep.md @@ -98,7 +98,7 @@ val result: DataStream[Alert] = patternStream.select(createAlert(_)) -Note that we use use Java 8 lambdas in our Java code examples to make them more succinct. +Note that we use Java 8 lambdas in our Java code examples to make them more succinct. ## The Pattern API @@ -521,10 +521,11 @@ def flatSelectFn(pattern : mutable.Map[String, IN], collector : Collector[OUT]) ### Handling Timed Out Partial Patterns -Whenever a pattern has a window length associated via the `within` key word, it is possible that partial event patterns will be discarded because they exceed the window length. -In order to react to these timeout events the `select` and `flatSelect` API calls allow to specify a timeout handler. +Whenever a pattern has a window length associated via the `within` keyword, it is possible that partial event patterns will be discarded because they exceed the window length. +In order to react to these timeout events the `select` and `flatSelect` API calls allow a timeout handler to be specified. This timeout handler is called for each partial event pattern which has timed out. -The timeout handler receives all so far matched events of the partial pattern and the timestamp when the timeout was detected. +The timeout handler receives all the events that have been matched so far by the pattern, and the timestamp when the timeout was detected. +
diff --git a/docs/dev/state.md b/docs/dev/state.md index ec8c5eb50ae29..37de0a8381230 100644 --- a/docs/dev/state.md +++ b/docs/dev/state.md @@ -73,20 +73,19 @@ active key (i.e. the key of the input element). It is important to keep in mind that these state objects are only used for interfacing with state. The state is not necessarily stored inside but might reside on disk or somewhere else. The second thing to keep in mind is that the value you get from the state -depend on the key of the input element. So the value you get in one invocation of your -user function can be different from the one you get in another invocation if the key of -the element is different. +depends on the key of the input element. So the value you get in one invocation of your +user function can differ from the value in another invocation if the keys involved are different. -To get a state handle you have to create a `StateDescriptor` this holds the name of the state +To get a state handle you have to create a `StateDescriptor`. This holds the name of the state (as we will later see you can create several states, and they have to have unique names so -that you can reference them), the type of the values that the state holds and possibly +that you can reference them), the type of the values that the state holds, and possibly a user-specified function, such as a `ReduceFunction`. Depending on what type of state you -want to retrieve you create one of `ValueStateDescriptor`, `ListStateDescriptor` or -`ReducingStateDescriptor`. +want to retrieve, you create either a `ValueStateDescriptor`, a `ListStateDescriptor` or +a `ReducingStateDescriptor`. State is accessed using the `RuntimeContext`, so it is only possible in *rich functions*. Please see [here]({{ site.baseurl }}/apis/common/#specifying-transformation-functions) for -information about that but we will also see an example shortly. The `RuntimeContext` that +information about that, but we will also see an example shortly. The `RuntimeContext` that is available in a `RichFunction` has these methods for accessing state: * `ValueState getState(ValueStateDescriptor)` @@ -147,7 +146,7 @@ env.fromElements(Tuple2.of(1L, 3L), Tuple2.of(1L, 5L), Tuple2.of(1L, 7L), Tuple2 This example implements a poor man's counting window. We key the tuples by the first field (in the example all have the same key `1`). The function stores the count and a running sum in -a `ValueState`, once the count reaches 2 it will emit the average and clear the state so that +a `ValueState`. Once the count reaches 2 it will emit the average and clear the state so that we start over from `0`. Note that this would keep a different state value for each different input key if we had tuples with different values in the first field. diff --git a/docs/dev/state_backends.md b/docs/dev/state_backends.md index e5b9c2a2f8737..70e472daf5730 100644 --- a/docs/dev/state_backends.md +++ b/docs/dev/state_backends.md @@ -41,16 +41,16 @@ chosen **State Backend**. Out of the box, Flink bundles these state backends: - - *MemoryStateBacked* + - *MemoryStateBackend* - *FsStateBackend* - *RocksDBStateBackend* -If nothing else is configured, the system will use the MemoryStateBacked. +If nothing else is configured, the system will use the MemoryStateBackend. ### The MemoryStateBackend -The *MemoryStateBacked* holds data internally as objects on the Java heap. Key/value state and window operators hold hash tables +The *MemoryStateBackend* holds data internally as objects on the Java heap. Key/value state and window operators hold hash tables that store the values, triggers, etc. Upon checkpoints, this state backend will snapshot the state and send it as part of the checkpoint acknowledgement messages to the diff --git a/docs/dev/windows.md b/docs/dev/windows.md index 67280dd704010..26118700f6e13 100644 --- a/docs/dev/windows.md +++ b/docs/dev/windows.md @@ -111,7 +111,7 @@ which we could process the aggregated elements. ### Tumbling Windows A *tumbling windows* assigner assigns elements to fixed length, non-overlapping windows of a -specified *window size*.. For example, if you specify a window size of 5 minutes, the window +specified *window size*. For example, if you specify a window size of 5 minutes, the window function will get 5 minutes worth of elements in each invocation. @@ -381,7 +381,7 @@ a concatenation of all the `Long` fields of the input. ### WindowFunction - The Generic Case -Using a `WindowFunction` provides most flexibility, at the cost of performance. The reason for this +Using a `WindowFunction` provides the most flexibility, at the cost of performance. The reason for this is that elements cannot be incrementally aggregated for a window and instead need to be buffered internally until the window is considered ready for processing. A `WindowFunction` gets an `Iterable` containing all the elements of the window being processed. The signature of diff --git a/docs/quickstart/run_example_quickstart.md b/docs/quickstart/run_example_quickstart.md index 70f87567de55e..e0792800b1bba 100644 --- a/docs/quickstart/run_example_quickstart.md +++ b/docs/quickstart/run_example_quickstart.md @@ -26,12 +26,12 @@ under the License. * This will be replaced by the TOC {:toc} -In this guide we will start from scratch and go from setting up a Flink project and running +In this guide we will start from scratch and go from setting up a Flink project to running a streaming analysis program on a Flink cluster. Wikipedia provides an IRC channel where all edits to the wiki are logged. We are going to read this channel in Flink and count the number of bytes that each user edits within -a given window of time. This is easy enough to implement in a few minutes using Flink but it will +a given window of time. This is easy enough to implement in a few minutes using Flink, but it will give you a good foundation from which to start building more complex analysis programs on your own. ## Setting up a Maven Project @@ -125,21 +125,21 @@ public class WikipediaAnalysis { } {% endhighlight %} -I admit it's very bare bones now but we will fill it as we go. Note, that I'll not give +The program is very basic now, but we will fill it in as we go. Note that I'll not give import statements here since IDEs can add them automatically. At the end of this section I'll show the complete code with import statements if you simply want to skip ahead and enter that in your editor. The first step in a Flink program is to create a `StreamExecutionEnvironment` (or `ExecutionEnvironment` if you are writing a batch job). This can be used to set execution -parameters and create sources for reading from external systems. So let's go ahead, add +parameters and create sources for reading from external systems. So let's go ahead and add this to the main method: {% highlight java %} StreamExecutionEnvironment see = StreamExecutionEnvironment.getExecutionEnvironment(); {% endhighlight %} -Next, we will create a source that reads from the Wikipedia IRC log: +Next we will create a source that reads from the Wikipedia IRC log: {% highlight java %} DataStream edits = see.addSource(new WikipediaEditsSource()); @@ -149,7 +149,7 @@ This creates a `DataStream` of `WikipediaEditEvent` elements that we can further the purposes of this example we are interested in determining the number of added or removed bytes that each user causes in a certain time window, let's say five seconds. For this we first have to specify that we want to key the stream on the user name, that is to say that operations -on this should take the key into account. In our case the summation of edited bytes in the windows +on this stream should take the user name into account. In our case the summation of edited bytes in the windows should be per unique user. For keying a Stream we have to provide a `KeySelector`, like this: {% highlight java %} @@ -165,8 +165,8 @@ KeyedStream keyedEdits = edits This gives us a Stream of `WikipediaEditEvent` that has a `String` key, the user name. We can now specify that we want to have windows imposed on this stream and compute a result based on elements in these windows. A window specifies a slice of a Stream -on which to perform a computation. They are required when performing an aggregation -computation on an infinite stream of elements. In our example we will say +on which to perform a computation. Windows are required when computing aggregations +on an infinite stream of elements. In our example we will say that we want to aggregate the sum of edited bytes for every five seconds: {% highlight java %} @@ -276,9 +276,10 @@ similar to this: The number in front of each line tells you on which parallel instance of the print sink the output was produced. -This should get you started with writing your own Flink programs. You can check out our guides -about [basic concepts]{{{ site.baseurl }}/apis/common/index.html} and the -[DataStream API]{{{ site.baseurl }}/apis/streaming/index.html} if you want to learn more. Stick +This should get you started with writing your own Flink programs. To learn more +you can check out our guides +about [basic concepts]({{ site.baseurl }}/apis/common/index.html) and the +[DataStream API]({{ site.baseurl }}/apis/streaming/index.html). Stick around for the bonus exercise if you want to learn about setting up a Flink cluster on your own machine and writing results to [Kafka](http://kafka.apache.org). From fe0c966b0b358095e8dc39c1d80ad71cc897b0f9 Mon Sep 17 00:00:00 2001 From: chobeat Date: Tue, 20 Sep 2016 16:27:16 +0200 Subject: [PATCH 065/299] [hotfix] [docs] Parametrized version of quickstart script and artifacts in the quickstart docs This closes #2522 --- docs/quickstart/java_api_quickstart.md | 2 +- docs/quickstart/run_example_quickstart.md | 2 +- docs/quickstart/scala_api_quickstart.md | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/quickstart/java_api_quickstart.md b/docs/quickstart/java_api_quickstart.md index 389c25a8f5608..1c7bd8a63ed9f 100644 --- a/docs/quickstart/java_api_quickstart.md +++ b/docs/quickstart/java_api_quickstart.md @@ -71,7 +71,7 @@ A note to Mac OS X users: The default JVM heapsize for Java is too small for Fli ## Build Project -If you want to __build your project__, go to your project directory and issue the `mvn clean install -Pbuild-jar` command. You will __find a jar__ that runs on every Flink cluster in __target/your-artifact-id-1.0-SNAPSHOT.jar__. There is also a fat-jar, __target/your-artifact-id-1.0-SNAPSHOT-flink-fat-jar.jar__. This +If you want to __build your project__, go to your project directory and issue the `mvn clean install -Pbuild-jar` command. You will __find a jar__ that runs on every Flink cluster in __target/your-artifact-id-{{ site.version }}.jar__. There is also a fat-jar, __target/your-artifact-id-{{ site.version }}-flink-fat-jar.jar__. This also contains all dependencies that get added to the maven project. ## Next Steps diff --git a/docs/quickstart/run_example_quickstart.md b/docs/quickstart/run_example_quickstart.md index e0792800b1bba..a64e87f4ec290 100644 --- a/docs/quickstart/run_example_quickstart.md +++ b/docs/quickstart/run_example_quickstart.md @@ -44,7 +44,7 @@ about this. For our purposes, the command to run is this: $ mvn archetype:generate\ -DarchetypeGroupId=org.apache.flink\ -DarchetypeArtifactId=flink-quickstart-java\ - -DarchetypeVersion=1.0.0\ + -DarchetypeVersion={{ site.version }}\ -DgroupId=wiki-edits\ -DartifactId=wiki-edits\ -Dversion=0.1\ diff --git a/docs/quickstart/scala_api_quickstart.md b/docs/quickstart/scala_api_quickstart.md index 16101d5f3e1a4..22562f470564a 100644 --- a/docs/quickstart/scala_api_quickstart.md +++ b/docs/quickstart/scala_api_quickstart.md @@ -171,7 +171,7 @@ The IntelliJ IDE also supports Maven and offers a plugin for Scala development. ### Build Project -If you want to __build your project__, go to your project directory and issue the `mvn clean package -Pbuild-jar` command. You will __find a jar__ that runs on every Flink cluster in __target/your-artifact-id-1.0-SNAPSHOT.jar__. There is also a fat-jar, __target/your-artifact-id-1.0-SNAPSHOT-flink-fat-jar.jar__. This +If you want to __build your project__, go to your project directory and issue the `mvn clean package -Pbuild-jar` command. You will __find a jar__ that runs on every Flink cluster in __target/your-artifact-id-{{ site.version }}.jar__. There is also a fat-jar, __target/your-artifact-id-{{ site.version }}-flink-fat-jar.jar__. This also contains all dependencies that get added to the maven project. ## Next Steps From 368e2c949903f264092abd8ddf0ba3bcd0b0ea21 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 20 Sep 2016 18:53:46 +0200 Subject: [PATCH 066/299] [hotfix] [streaming api] Add proper deprecation JavaDocs Also includes minor style cleanup of a test. --- .../environment/StreamExecutionEnvironment.java | 14 ++++++++++++-- .../apache/flink/streaming/api/TypeFillTest.java | 10 +++++----- 2 files changed, 17 insertions(+), 7 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java index 78aab970d71e5..06823d57b911e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java @@ -577,8 +577,8 @@ public & Serializable>void registerTypeWithKryoSerializ * @param serializerClass * The class of the serializer to use. */ - public void registerTypeWithKryoSerializer(Class type, - Class> serializerClass) { + public void registerTypeWithKryoSerializer(Class type, + Class> serializerClass) { config.registerTypeWithKryoSerializer(type, serializerClass); } @@ -1094,8 +1094,11 @@ public DataStreamSource readFile(FileInputFormat inputFormat, * contents * of files. * @return The DataStream containing the given directory. + * + * @deprecated Use {@link #readFile(FileInputFormat, String, FileProcessingMode, long)} instead. */ @Deprecated + @SuppressWarnings("deprecation") public DataStream readFileStream(String filePath, long intervalMillis, FileMonitoringFunction.WatchType watchType) { DataStream> source = addSource(new FileMonitoringFunction( @@ -1169,6 +1172,8 @@ public DataStreamSource readFile(FileInputFormat inputFormat, * while * a negative value ensures retrying forever. * @return A data stream containing the strings received from the socket + * + * @deprecated Use {@link #socketTextStream(String, int, String, long)} instead. */ @Deprecated public DataStreamSource socketTextStream(String hostname, int port, char delimiter, long maxRetry) { @@ -1215,8 +1220,11 @@ public DataStreamSource socketTextStream(String hostname, int port, Stri * @param delimiter * A character which splits received strings into records * @return A data stream containing the strings received from the socket + * + * @deprecated Use {@link #socketTextStream(String, int, String)} instead. */ @Deprecated + @SuppressWarnings("deprecation") public DataStreamSource socketTextStream(String hostname, int port, char delimiter) { return socketTextStream(hostname, port, delimiter, 0); } @@ -1313,7 +1321,9 @@ public DataStreamSource createInput(InputFormat inputFormat, DataStreamSource source; if (inputFormat instanceof FileInputFormat) { + @SuppressWarnings("unchecked") FileInputFormat format = (FileInputFormat) inputFormat; + source = createFileInput(format, typeInfo, "Custom File source", FileProcessingMode.PROCESS_ONCE, -1); } else { diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/TypeFillTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/TypeFillTest.java index d931f7b4a4ab9..a4abbcba442b1 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/TypeFillTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/TypeFillTest.java @@ -96,7 +96,7 @@ public String map(Long value) throws Exception { } - private class TestSource implements SourceFunction { + private static class TestSource implements SourceFunction { private static final long serialVersionUID = 1L; @Override @@ -106,19 +106,19 @@ public void run(SourceContext ctx) throws Exception {} public void cancel() {} } - private class TestMap implements MapFunction { + private static class TestMap implements MapFunction { @Override public O map(T value) throws Exception { return null; } } - private class TestFlatMap implements FlatMapFunction { + private static class TestFlatMap implements FlatMapFunction { @Override public void flatMap(T value, Collector out) throws Exception {} } - private class TestCoMap implements CoMapFunction { + private static class TestCoMap implements CoMapFunction { @Override public OUT map1(IN1 value) { @@ -132,7 +132,7 @@ public OUT map2(IN2 value) { } - private class TestCoFlatMap implements CoFlatMapFunction { + private static class TestCoFlatMap implements CoFlatMapFunction { @Override public void flatMap1(IN1 value, Collector out) throws Exception {} From 5f71771bd7a817557a135088d021046941b4cde7 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 20 Sep 2016 19:47:49 +0200 Subject: [PATCH 067/299] [FLINK-4645] [core] Adjust signatures of 'registerTypeWithKryoSerializer(...)' methods to allow simpler passing of classes without generic casting. This is not API breaking due to generic type erasure in Java. The changes method still has the same binary signature. --- .../org/apache/flink/api/common/ExecutionConfig.java | 11 ++++++++--- .../api/environment/StreamExecutionEnvironment.java | 4 ++-- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java index 81ee930b35e38..aadf867930279 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java @@ -311,6 +311,7 @@ public void setRestartStrategy(RestartStrategies.RestartStrategyConfiguration re * @return The specified restart configuration */ @PublicEvolving + @SuppressWarnings("deprecation") public RestartStrategies.RestartStrategyConfiguration getRestartStrategy() { if (restartStrategyConfiguration == null) { // support the old API calls by creating a restart strategy from them @@ -601,11 +602,15 @@ public & Serializable>void registerTypeWithKryoSerializ * @param type The class of the types serialized with the given serializer. * @param serializerClass The class of the serializer to use. */ - public void registerTypeWithKryoSerializer(Class type, Class> serializerClass) { + @SuppressWarnings("rawtypes") + public void registerTypeWithKryoSerializer(Class type, Class serializerClass) { if (type == null || serializerClass == null) { throw new NullPointerException("Cannot register null class or serializer."); } - registeredTypesWithKryoSerializerClasses.put(type, serializerClass); + + @SuppressWarnings("unchecked") + Class> castedSerializerClass = (Class>) serializerClass; + registeredTypesWithKryoSerializerClasses.put(type, castedSerializerClass); } /** @@ -620,7 +625,7 @@ public void registerPojoType(Class type) { if (type == null) { throw new NullPointerException("Cannot register null type class."); } - if(!registeredPojoTypes.contains(type)) { + if (!registeredPojoTypes.contains(type)) { registeredPojoTypes.add(type); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java index 06823d57b911e..8fc2872e7969b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java @@ -577,8 +577,8 @@ public & Serializable>void registerTypeWithKryoSerializ * @param serializerClass * The class of the serializer to use. */ - public void registerTypeWithKryoSerializer(Class type, - Class> serializerClass) { + @SuppressWarnings("rawtypes") + public void registerTypeWithKryoSerializer(Class type, Class serializerClass) { config.registerTypeWithKryoSerializer(type, serializerClass); } From cfe1dc3474ec72150a4f24aec666e0fcaa6160d7 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 20 Sep 2016 20:49:40 +0200 Subject: [PATCH 068/299] [FLINK-4640] [streaming api] Ensure that the state descriptors properly initialize the serializers. --- .../api/datastream/AllWindowedStream.java | 4 +- .../api/datastream/WindowedStream.java | 4 +- .../operators/windowing/WindowOperator.java | 43 ++-- .../operators/StateDescriptorPassingTest.java | 214 ++++++++++++++++++ .../windowing/WindowOperatorTest.java | 2 + 5 files changed, 241 insertions(+), 26 deletions(-) create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StateDescriptorPassingTest.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java index 4b083c89df8f5..6b09f3c4abe16 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/AllWindowedStream.java @@ -469,9 +469,7 @@ public SingleOutputStreamOperator apply(R initialValue, FoldFunction stateDesc = new FoldingStateDescriptor<>("window-contents", - initialValue, - foldFunction, - resultType); + initialValue, foldFunction, resultType.createSerializer(getExecutionEnvironment().getConfig())); opName = "TriggerWindow(" + windowAssigner + ", " + stateDesc + ", " + trigger + ", " + udfName + ")"; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java index e81d7afcff40f..ae9861904405e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java @@ -489,9 +489,7 @@ public SingleOutputStreamOperator apply(R initialValue, FoldFunction stateDesc = new FoldingStateDescriptor<>("window-contents", - initialValue, - foldFunction, - resultType); + initialValue, foldFunction, resultType.createSerializer(getExecutionEnvironment().getConfig())); opName = "TriggerWindow(" + windowAssigner + ", " + stateDesc + ", " + trigger + ", " + udfName + ")"; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java index 25ec5193420ca..dffa2a1bbee0b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java @@ -61,7 +61,6 @@ import org.apache.flink.streaming.runtime.operators.Triggerable; import org.apache.flink.streaming.runtime.operators.windowing.functions.InternalWindowFunction; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.util.Preconditions; import java.io.IOException; import java.io.ObjectInputStream; @@ -74,7 +73,8 @@ import java.util.Set; import java.util.concurrent.ScheduledFuture; -import static java.util.Objects.requireNonNull; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; /** * An operator that implements the logic for windowing based on a {@link WindowAssigner} and @@ -186,26 +186,29 @@ public class WindowOperator /** * Creates a new {@code WindowOperator} based on the given policies and user functions. */ - public WindowOperator(WindowAssigner windowAssigner, - TypeSerializer windowSerializer, - KeySelector keySelector, - TypeSerializer keySerializer, - StateDescriptor, ?> windowStateDescriptor, - InternalWindowFunction windowFunction, - Trigger trigger, - long allowedLateness) { + public WindowOperator( + WindowAssigner windowAssigner, + TypeSerializer windowSerializer, + KeySelector keySelector, + TypeSerializer keySerializer, + StateDescriptor, ?> windowStateDescriptor, + InternalWindowFunction windowFunction, + Trigger trigger, + long allowedLateness) { super(windowFunction); - this.windowAssigner = requireNonNull(windowAssigner); - this.windowSerializer = windowSerializer; - this.keySelector = requireNonNull(keySelector); - this.keySerializer = requireNonNull(keySerializer); + checkArgument(allowedLateness >= 0); - this.windowStateDescriptor = windowStateDescriptor; - this.trigger = requireNonNull(trigger); + checkArgument(windowStateDescriptor == null || windowStateDescriptor.isSerializerInitialized(), + "window state serializer is not properly initialized"); - Preconditions.checkArgument(allowedLateness >= 0); + this.windowAssigner = checkNotNull(windowAssigner); + this.windowSerializer = checkNotNull(windowSerializer); + this.keySelector = checkNotNull(keySelector); + this.keySerializer = checkNotNull(keySerializer); + this.windowStateDescriptor = windowStateDescriptor; + this.trigger = checkNotNull(trigger); this.allowedLateness = allowedLateness; setChainingStrategy(ChainingStrategy.ALWAYS); @@ -666,7 +669,7 @@ public long getCurrentWatermark() { public ValueState getKeyValueState(String name, Class stateType, S defaultState) { - requireNonNull(stateType, "The state type class must not be null"); + checkNotNull(stateType, "The state type class must not be null"); TypeInformation typeInfo; try { @@ -686,8 +689,8 @@ public ValueState getKeyValueState(String name, TypeInformation stateType, S defaultState) { - requireNonNull(name, "The name of the state must not be null"); - requireNonNull(stateType, "The state type information must not be null"); + checkNotNull(name, "The name of the state must not be null"); + checkNotNull(stateType, "The state type information must not be null"); ValueStateDescriptor stateDesc = new ValueStateDescriptor<>(name, stateType.createSerializer(getExecutionConfig()), defaultState); return getPartitionedState(stateDesc); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StateDescriptorPassingTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StateDescriptorPassingTest.java new file mode 100644 index 0000000000000..c0ca6a00367d7 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StateDescriptorPassingTest.java @@ -0,0 +1,214 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.api.operators; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.serializers.JavaSerializer; + +import org.apache.flink.api.common.functions.FoldFunction; +import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.common.state.StateDescriptor; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; +import org.apache.flink.streaming.api.TimeCharacteristic; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; +import org.apache.flink.streaming.api.functions.windowing.WindowFunction; +import org.apache.flink.streaming.api.transformations.OneInputTransformation; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.streaming.runtime.operators.windowing.WindowOperator; +import org.apache.flink.util.Collector; + +import org.junit.Test; + +import java.io.File; + +import static org.junit.Assert.*; + +/** + * Various tests around the proper passing of state descriptors to the operators + * and their serialization. + * + * The tests use an arbitrary generic type to validate the behavior. + */ +@SuppressWarnings("serial") +public class StateDescriptorPassingTest { + + @Test + public void testFoldWindowState() throws Exception { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime); + env.registerTypeWithKryoSerializer(File.class, JavaSerializer.class); + + DataStream src = env.fromElements("abc"); + + SingleOutputStreamOperator result = src + .keyBy(new KeySelector() { + @Override + public String getKey(String value) { + return null; + } + }) + .timeWindow(Time.milliseconds(1000)) + .fold(new File("/"), new FoldFunction() { + + @Override + public File fold(File a, String e) { + return null; + } + }); + + validateStateDescriptorConfigured(result); + } + + @Test + public void testReduceWindowState() throws Exception { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime); + env.registerTypeWithKryoSerializer(File.class, JavaSerializer.class); + + DataStream src = env.fromElements(new File("/")); + + SingleOutputStreamOperator result = src + .keyBy(new KeySelector() { + @Override + public String getKey(File value) { + return null; + } + }) + .timeWindow(Time.milliseconds(1000)) + .reduce(new ReduceFunction() { + + @Override + public File reduce(File value1, File value2) { + return null; + } + }); + + validateStateDescriptorConfigured(result); + } + + @Test + public void testApplyWindowState() throws Exception { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime); + env.registerTypeWithKryoSerializer(File.class, JavaSerializer.class); + + DataStream src = env.fromElements(new File("/")); + + SingleOutputStreamOperator result = src + .keyBy(new KeySelector() { + @Override + public String getKey(File value) { + return null; + } + }) + .timeWindow(Time.milliseconds(1000)) + .apply(new WindowFunction() { + @Override + public void apply(String s, TimeWindow window, + Iterable input, Collector out) {} + }); + + validateStateDescriptorConfigured(result); + } + + @Test + public void testFoldWindowAllState() throws Exception { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime); + env.registerTypeWithKryoSerializer(File.class, JavaSerializer.class); + + DataStream src = env.fromElements("abc"); + + SingleOutputStreamOperator result = src + .timeWindowAll(Time.milliseconds(1000)) + .fold(new File("/"), new FoldFunction() { + + @Override + public File fold(File a, String e) { + return null; + } + }); + + validateStateDescriptorConfigured(result); + } + + @Test + public void testReduceWindowAllState() throws Exception { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime); + env.registerTypeWithKryoSerializer(File.class, JavaSerializer.class); + + DataStream src = env.fromElements(new File("/")); + + SingleOutputStreamOperator result = src + .timeWindowAll(Time.milliseconds(1000)) + .reduce(new ReduceFunction() { + + @Override + public File reduce(File value1, File value2) { + return null; + } + }); + + validateStateDescriptorConfigured(result); + } + + @Test + public void testApplyWindowAllState() throws Exception { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime); + env.registerTypeWithKryoSerializer(File.class, JavaSerializer.class); + + DataStream src = env.fromElements(new File("/")); + + SingleOutputStreamOperator result = src + .timeWindowAll(Time.milliseconds(1000)) + .apply(new AllWindowFunction() { + @Override + public void apply(TimeWindow window, Iterable input, Collector out) {} + }); + + validateStateDescriptorConfigured(result); + } + + // ------------------------------------------------------------------------ + // generic validation + // ------------------------------------------------------------------------ + + private void validateStateDescriptorConfigured(SingleOutputStreamOperator result) { + OneInputTransformation transform = (OneInputTransformation) result.getTransformation(); + WindowOperator op = (WindowOperator) transform.getOperator(); + StateDescriptor descr = op.getStateDescriptor(); + + // this would be the first statement to fail if state descriptors were not properly initialized + TypeSerializer serializer = descr.getSerializer(); + assertTrue(serializer instanceof KryoSerializer); + + Kryo kryo = ((KryoSerializer) serializer).getKryo(); + + assertTrue("serializer registration was not properly passed on", + kryo.getSerializer(File.class) instanceof JavaSerializer); + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java index 67a6f556796c5..fd73bcc584350 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java @@ -2102,6 +2102,7 @@ public Tuple2 fold(Tuple2 accumulator, Tuple2< } }, inputType); + windowStateDesc.initializeSerializerUnlessSet(new ExecutionConfig()); WindowOperator, Tuple2, Tuple2, TimeWindow> operator = new WindowOperator<>( @@ -2246,6 +2247,7 @@ public Tuple2 fold(Tuple2 accumulator, Tuple2< } }, inputType); + windowStateDesc.initializeSerializerUnlessSet(new ExecutionConfig()); WindowOperator, Tuple2, Tuple2, TimeWindow> operator = new WindowOperator<>( From ebe35b6e70b2e6158b8e791d477597b555adbee9 Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Mon, 19 Sep 2016 17:54:23 +0200 Subject: [PATCH 069/299] [hotfix] [tests] Fix race condition in RescalingITCase that could make the test stuck in a blocking call until timeout This closes #2513 --- .../test/checkpointing/RescalingITCase.java | 51 +++++++++++-------- 1 file changed, 30 insertions(+), 21 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java index 7f1d7f3360f78..263bf790eb042 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java @@ -69,9 +69,9 @@ public class RescalingITCase extends TestLogger { - private static int numTaskManagers = 2; - private static int slotsPerTaskManager = 2; - private static int numSlots = numTaskManagers * slotsPerTaskManager; + private static final int numTaskManagers = 2; + private static final int slotsPerTaskManager = 2; + private static final int numSlots = numTaskManagers * slotsPerTaskManager; private static TestingCluster cluster; @@ -109,12 +109,12 @@ public static void teardown() { */ @Test public void testSavepointRescalingWithPartitionedState() throws Exception { - int numberKeys = 42; - int numberElements = 1000; - int numberElements2 = 500; - int parallelism = numSlots / 2; - int parallelism2 = numSlots; - int maxParallelism = 13; + final int numberKeys = 42; + final int numberElements = 1000; + final int numberElements2 = 500; + final int parallelism = numSlots / 2; + final int parallelism2 = numSlots; + final int maxParallelism = 13; FiniteDuration timeout = new FiniteDuration(3, TimeUnit.MINUTES); Deadline deadline = timeout.fromNow(); @@ -214,9 +214,9 @@ public void testSavepointRescalingWithPartitionedState() throws Exception { */ @Test public void testSavepointRescalingFailureWithNonPartitionedState() throws Exception { - int parallelism = numSlots / 2; - int parallelism2 = numSlots; - int maxParallelism = 13; + final int parallelism = numSlots / 2; + final int parallelism2 = numSlots; + final int maxParallelism = 13; FiniteDuration timeout = new FiniteDuration(3, TimeUnit.MINUTES); Deadline deadline = timeout.fromNow(); @@ -235,12 +235,14 @@ public void testSavepointRescalingFailureWithNonPartitionedState() throws Except Object savepointResponse = null; - // we might be too early for taking a savepoint if the operators have not been started yet + // wait until the operator is started + NonPartitionedStateSource.workStartedLatch.await(); + while (deadline.hasTimeLeft()) { Future savepointPathFuture = jobManager.ask(new JobManagerMessages.TriggerSavepoint(jobID), deadline.timeLeft()); - - savepointResponse = Await.result(savepointPathFuture, deadline.timeLeft()); + FiniteDuration waitingTime = new FiniteDuration(10, TimeUnit.SECONDS); + savepointResponse = Await.result(savepointPathFuture, waitingTime); if (savepointResponse instanceof JobManagerMessages.TriggerSavepointSuccess) { break; @@ -428,6 +430,8 @@ private static JobGraph createNonPartitionedStateJobGraph(int parallelism, int m env.enableCheckpointing(checkpointInterval); env.setRestartStrategy(RestartStrategies.noRestart()); + NonPartitionedStateSource.workStartedLatch = new CountDownLatch(1); + DataStream input = env.addSource(new NonPartitionedStateSource()); input.addSink(new DiscardingSink()); @@ -466,7 +470,7 @@ public Integer getKey(Integer value) throws Exception { DataStream> result = input.flatMap(new SubtaskIndexFlatMapper(numberElements)); - result.addSink(new CollectionSink()); + result.addSink(new CollectionSink>()); return env.getStreamGraph().getJobGraph(); } @@ -504,7 +508,7 @@ public Integer getKey(Integer value) throws Exception { DataStream> result = input.flatMap(new SubtaskIndexFlatMapper(numberElements)); - result.addSink(new CollectionSink()); + result.addSink(new CollectionSink>()); return env.getStreamGraph().getJobGraph(); } @@ -645,8 +649,10 @@ private static class NonPartitionedStateSource extends RichParallelSourceFunctio private static final long serialVersionUID = -8108185918123186841L; - private int counter = 0; - private boolean running = true; + private static volatile CountDownLatch workStartedLatch = new CountDownLatch(1); + + private volatile int counter = 0; + private volatile boolean running = true; @Override public Integer snapshotState(long checkpointId, long checkpointTimestamp) throws Exception { @@ -669,13 +675,16 @@ public void run(SourceContext ctx) throws Exception { ctx.collect(counter * getRuntimeContext().getIndexOfThisSubtask()); } - Thread.sleep(100); + Thread.sleep(2); + if(counter == 10) { + workStartedLatch.countDown(); + } } } @Override public void cancel() { - running = true; + running = false; } } } From 1b22a42ea8c5efbd711a9678eeb303310466610a Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 20 Sep 2016 23:10:44 +0200 Subject: [PATCH 070/299] [docs] Update docs on data types and serialization, to include type hints, type registration, and serializer registration. --- docs/dev/types_serialization.md | 153 +++++++++++++++++++++----------- 1 file changed, 102 insertions(+), 51 deletions(-) diff --git a/docs/dev/types_serialization.md b/docs/dev/types_serialization.md index 8a32491c4137d..4b8e25f6453d3 100644 --- a/docs/dev/types_serialization.md +++ b/docs/dev/types_serialization.md @@ -1,5 +1,5 @@ --- -title: "Data Types" +title: "Data Types & Serialization" nav-id: types nav-parent_id: dev nav-pos: 9 @@ -23,13 +23,8 @@ specific language governing permissions and limitations under the License. --> -Flink handles types in a unique way, containing its own type descriptors, -generic type extraction, and type serialization framework. -This document describes the concepts and the rationale behind them. - -There are fundamental differences in the way that the Scala API and -the Java API handle type information, so most of the issues described -here relate only to one of the to APIs. +Apache Flink handles data types and serialization in a unique way, containing its own type descriptors, +generic type extraction, and type serialization framework. This document describes the concepts and the rationale behind them. * This will be replaced by the TOC {:toc} @@ -37,21 +32,43 @@ here relate only to one of the to APIs. ## Type handling in Flink -Flink tries to know as much information about what types enter and leave user functions as possible. -This stands in contrast to the approach to just assuming nothing and letting the -programming language and serialization framework handle all types dynamically. +Flink tries to infer a lot of information about the data types that are exchanged and stored during the distributed computation. +Think about it like a database that infers the schema of tables. In most cases, Flink infers all necessary information seamlessly +by itself. Having the type information allows Flink to do some cool things: + +* Using POJOs types and grouping / joining / aggregating them by referring to field names (like `dataSet.keyBy("username")`). + The type information allows Flink to check (for typos and type compatibility) early rather than failing later ar runtime. + +* The more Flink knows about data types, the better the serialization and data layout schemes are. + That is quite important for the memory usage paradigm in Flink (work on serialized data inside/outside the heap where ever possible + and make serialization very cheap). + +* Finally, it also spares users in the majority of cases from worrying about serialization frameworks and having to register types. + +In general, the information about data types is needed during the *pre-flight phase* - that is, when the program's calls on `DataStream` +and `DataSet` are made, and before any call to `execute()`, `print()`, `count()`, or `collect()`. + + +## Most Frequent Issues + +The most frequent issues where users need to interact with Flink's data type handling are: -* To allow using POJOs and grouping/joining them by referring to field names, Flink needs the type - information to make checks (for typos and type compatibility) before the job is executed. +* **Registering subtypes:** If the function signatures describe only the supertypes, but they actually use subtypes of those during execution, + it may increase performance a lot to make Flink aware of these subtypes. + For that, call `.registerType(clazz)` on the `StreamExecutionEnvironment` or `ExecutionEnvironment` for each subtype. -* The more we know, the better serialization and data layout schemes the compiler/optimizer can develop. - That is quite important for the memory usage paradigm in Flink (work on serialized data - inside/outside the heap and make serialization very cheap). +* **Registering custom serializers:** Flink falls back to [Kryo](https://github.com/EsotericSoftware/kryo) for the types that it does not handle transparently + by itself. Not all types are seamlessly handled by Kryo (and thus by Flink). For example, many Google Guava collection types do not work well + by default. The solution is to register additional serializers for the types that cause problems. + Call `.getConfig().addDefaultKryoSerializer(clazz, serializer)` on the `StreamExecutionEnvironment` or `ExecutionEnvironment`. + Additional Kryo serializers are available in many libraries. -* For the upcoming logical programs (see roadmap draft) we need this to know the "schema" of functions. +* **Adding Type Hints:** Sometimes, when Flink cannot infer the generic types despits all tricks, a user must pass a *type hint*. That is generally + only necessary in the Java API. The [Type Hints Section](#type-hints-in-the-java-api) describes that in more detail. -* Finally, it also spares users having to worry about serialization frameworks and having to register - types at those frameworks. +* **Manually creating a `TypeInformation`:** This may be necessary for some API calls where it is not possible for Flink to infer + the data types due to Java's generic type erasure. See [Creating a TypeInformation or TypeSerializer](#creating-a-typeinformation-or-typeserializer) + for details. ## Flink's TypeInformation class @@ -75,7 +92,7 @@ Internally, Flink makes the following distinctions between types: * POJOs: classes that follow a certain bean-like pattern -* Scala auxiliary types (Option, Either, Lists, Maps, ...) +* Auxiliary types (Option, Either, Lists, Maps, ...) * Generic types: These will not be serialized by Flink itself, but by Kryo. @@ -84,18 +101,66 @@ names in the definition of keys: `dataSet.join(another).where("name").equalTo("p They are also transparent to the runtime and can be handled very efficiently by Flink. -**Rules for POJO types** +#### Rules for POJO types Flink recognizes a data type as a POJO type (and allows "by-name" field referencing) if the following conditions are fulfilled: * The class is public and standalone (no non-static inner class) * The class has a public no-argument constructor -* All fields in the class (and all superclasses) are either public or - or have a public getter and a setter method that follows the Java beans +* All fields in the class (and all superclasses) are either public (and non-final) + or have a public getter- and a setter- method that follows the Java beans naming conventions for getters and setters. +#### Creating a TypeInformation or TypeSerializer + +To create a TypeInformation object for a type, use the language specific way: + +
+
+Because Java generally erases generic type information, you need to pass the type to the TypeInformation +construction: + +For non-generic types, you can pass the Class: +{% highlight java %} +TypeInformation info = TypeInformation.of(String.class); +{% endhighlight %} + +For generic types, you need to "capture" the generic type information via the `TypeHint`: +{% highlight java %} +TypeInformation> info = TypeInformation.of(new TypeHint>(){}); +{% endhighlight %} +Internally, this creates an anonymous subclass of the TypeHint that captures the generic information to preserve it +until runtime. +
+ +
+In Scala, Flink uses *macros* that runs at compile time and captures all generic type information while it is +still available. +{% highlight scala %} +// important: this import is needed to access the 'createTypeInformation' macro function +import org.apache.flink.streaming.api.scala._ + +val stringInfo: TypeInformation[String] = createTypeInformation[String] + +val tupleInfo: TypeInformation[(String, Double)] = createTypeInformation[(String, Double)] +{% endhighlight %} + +You can still use the same method as in Java as a fallback. +
+
+ +To create a `TypeSerializer`, simply call `typeInfo.createSerializer(config)` on the `TypeInformation` object. + +The `config` parameter is of type `ExecutionConfig` and holds the information about the program's registered +custom serializers. Where ever possibly, try to pass the programs proper ExecutionConfig. You can usually +obtain it from `DataStream` or `DataSet` via calling `getExecutionConfig()`. Inside functions (like `MapFunction`), you can +get it by making the function a [Rich Function]() and calling `getRuntimeContext().getExecutionConfig()`. + +-------- +-------- + ## Type Information in the Scala API Scala has very elaborate concepts for runtime type information though *type manifests* and *class tags*. In @@ -156,15 +221,15 @@ def selectFirst[T : TypeInformation](input: DataSet[(T, _)]) : DataSet[T] = { {% endhighlight %} +-------- +-------- -## Type Information in the Java API -Java in general erases generic type information. Only for subclasses of generic classes, the subclass -stores the type to which the generic type variables bind. +## Type Information in the Java API -Flink uses reflection on the (anonymous) classes that implement the user functions to figure out the types of -the generic parameters of the function. This logic also contains some simple type inference for cases where -the return types of functions are dependent on input types, such as in the generic utility method below: +In the general case, Java erases generic type information. Flink tries to reconstruct as much type information +as possible via reflection, using the few bits that Java preserves (mainly function signatures and subclass information). +This logic also contains some simple type inference for cases where the return type of a function depends on its input type: {% highlight java %} public class AppendOne extends MapFunction> { @@ -175,16 +240,14 @@ public class AppendOne extends MapFunction> { } {% endhighlight %} -Not in all cases can Flink figure out the data types of functions reliably in Java. -Some issues remain with generic lambdas (we are trying to solve this with the Java community, -see below) and with generic type variables that we cannot infer. +There are cases where Flink cannot reconstruct all generic type information. In that case, a user has to help out via *type hints*. #### Type Hints in the Java API -To help cases where Flink cannot reconstruct the erased generic type information, the Java API -offers so called *type hints* from version 0.9 on. The type hints tell the system the type of -the data set produced by a function. The following gives an example: +In cases where Flink cannot reconstruct the erased generic type information, the Java API +offers so called *type hints*. The type hints tell the system the type of +the data stream or data set produced by a function: {% highlight java %} DataSet result = dataSet @@ -193,12 +256,11 @@ DataSet result = dataSet {% endhighlight %} The `returns` statement specifies the produced type, in this case via a class. The hints support -type definition through +type definition via * Classes, for non-parameterized types (no generics) -* Strings in the form of `returns("Tuple2")`, which are parsed and converted - to a TypeInformation. -* A TypeInformation directly +* TypeHints in the form of `returns(new TypeHint>(){})`. The `TypeHint` class + can capture generic type information and preserve it for the runtime (via an anonymous subclass). #### Type extraction for Java 8 lambdas @@ -208,18 +270,7 @@ with an implementing class that extends the function interface. Currently, Flink tries to figure out which method implements the lambda and uses Java's generic signatures to determine the parameter types and the return type. However, these signatures are not generated for lambdas -by all compilers (as of writing this document only reliably by the Eclipse JDT compiler 4.5 from Milestone 2 -onwards) - - -**Improving Type information for Java Lambdas** - -One of the Flink committers (Timo Walther) has actually become active in the Eclipse JDT compiler community and -in the OpenJDK community and submitted patches to the compiler to improve availability of type information -available for Java 8 lambdas. - -The Eclipse JDT compiler has added support for this as of version 4.5 M4. Discussion about the feature in the -OpenJDK compiler is pending. +by all compilers (as of writing this document only reliably by the Eclipse JDT compiler from 4.5 onwards). #### Serialization of POJO types From 8ac887e5b0f8d17680ffd1a5f583b32015be8bd3 Mon Sep 17 00:00:00 2001 From: twalthr Date: Wed, 27 Jul 2016 14:51:07 +0200 Subject: [PATCH 071/299] [FLINK-4248] [core] [table] CsvTableSource does not support reading SqlTimeTypeInfo types This closes #2303. --- .../flink/types/parser/BigDecParser.java | 31 ++--- .../flink/types/parser/BigIntParser.java | 44 +++---- .../flink/types/parser/DoubleParser.java | 44 +++---- .../flink/types/parser/DoubleValueParser.java | 27 ++--- .../flink/types/parser/FieldParser.java | 48 ++++++++ .../flink/types/parser/FloatParser.java | 48 +++----- .../flink/types/parser/FloatValueParser.java | 27 ++--- .../flink/types/parser/SqlDateParser.java | 108 ++++++++++++++++++ .../flink/types/parser/SqlTimeParser.java | 102 +++++++++++++++++ .../types/parser/SqlTimestampParser.java | 108 ++++++++++++++++++ .../typeutils/base/SqlTimeComparatorTest.java | 2 +- .../typeutils/base/SqlTimeSerializerTest.java | 2 +- .../base/SqlTimestampComparatorTest.java | 6 +- .../base/SqlTimestampSerializerTest.java | 6 +- .../flink/types/parser/SqlDateParserTest.java | 64 +++++++++++ .../flink/types/parser/SqlTimeParserTest.java | 63 ++++++++++ .../types/parser/SqlTimestampParserTest.java | 69 +++++++++++ .../runtime/io/RowCsvInputFormatTest.scala | 42 ++++++- 18 files changed, 675 insertions(+), 166 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/types/parser/SqlDateParser.java create mode 100644 flink-core/src/main/java/org/apache/flink/types/parser/SqlTimeParser.java create mode 100644 flink-core/src/main/java/org/apache/flink/types/parser/SqlTimestampParser.java create mode 100644 flink-core/src/test/java/org/apache/flink/types/parser/SqlDateParserTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/types/parser/SqlTimeParserTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/types/parser/SqlTimestampParserTest.java diff --git a/flink-core/src/main/java/org/apache/flink/types/parser/BigDecParser.java b/flink-core/src/main/java/org/apache/flink/types/parser/BigDecParser.java index 46a07fa041451..9c9f57f549045 100644 --- a/flink-core/src/main/java/org/apache/flink/types/parser/BigDecParser.java +++ b/flink-core/src/main/java/org/apache/flink/types/parser/BigDecParser.java @@ -35,42 +35,27 @@ public class BigDecParser extends FieldParser { @Override public int parseField(byte[] bytes, int startPos, int limit, byte[] delimiter, BigDecimal reusable) { - int i = startPos; - - final int delimLimit = limit - delimiter.length + 1; - - while (i < limit) { - if (i < delimLimit && delimiterNext(bytes, i, delimiter)) { - if (i == startPos) { - setErrorState(ParseErrorState.EMPTY_COLUMN); - return -1; - } - break; - } - i++; - } - - if (i > startPos && - (Character.isWhitespace(bytes[startPos]) || Character.isWhitespace(bytes[(i - 1)]))) { - setErrorState(ParseErrorState.NUMERIC_VALUE_ILLEGAL_CHARACTER); + final int endPos = nextStringEndPos(bytes, startPos, limit, delimiter); + if (endPos < 0) { return -1; } try { - final int length = i - startPos; + final int length = endPos - startPos; if (reuse == null || reuse.length < length) { reuse = new char[length]; } for (int j = 0; j < length; j++) { final byte b = bytes[startPos + j]; if ((b < '0' || b > '9') && b != '-' && b != '+' && b != '.' && b != 'E' && b != 'e') { - throw new NumberFormatException(); + setErrorState(ParseErrorState.NUMERIC_VALUE_ILLEGAL_CHARACTER); + return -1; } reuse[j] = (char) bytes[startPos + j]; } this.result = new BigDecimal(reuse, 0, length); - return (i == limit) ? limit : i + delimiter.length; + return (endPos == limit) ? limit : endPos + delimiter.length; } catch (NumberFormatException e) { setErrorState(ParseErrorState.NUMERIC_VALUE_FORMAT_ERROR); return -1; @@ -96,7 +81,7 @@ public BigDecimal getLastResult() { * @param startPos The offset to start the parsing. * @param length The length of the byte sequence (counting from the offset). * @return The parsed value. - * @throws NumberFormatException Thrown when the value cannot be parsed because the text + * @throws IllegalArgumentException Thrown when the value cannot be parsed because the text * represents not a correct number. */ public static final BigDecimal parseField(byte[] bytes, int startPos, int length) { @@ -113,7 +98,7 @@ public static final BigDecimal parseField(byte[] bytes, int startPos, int length * @param length The length of the byte sequence (counting from the offset). * @param delimiter The delimiter that terminates the field. * @return The parsed value. - * @throws NumberFormatException Thrown when the value cannot be parsed because the text + * @throws IllegalArgumentException Thrown when the value cannot be parsed because the text * represents not a correct number. */ public static final BigDecimal parseField(byte[] bytes, int startPos, int length, char delimiter) { diff --git a/flink-core/src/main/java/org/apache/flink/types/parser/BigIntParser.java b/flink-core/src/main/java/org/apache/flink/types/parser/BigIntParser.java index 13361c176b961..11e459ab7d25c 100644 --- a/flink-core/src/main/java/org/apache/flink/types/parser/BigIntParser.java +++ b/flink-core/src/main/java/org/apache/flink/types/parser/BigIntParser.java @@ -34,31 +34,21 @@ public class BigIntParser extends FieldParser { @Override public int parseField(byte[] bytes, int startPos, int limit, byte[] delimiter, BigInteger reusable) { - int i = startPos; - - final int delimLimit = limit - delimiter.length + 1; - - while (i < limit) { - if (i < delimLimit && delimiterNext(bytes, i, delimiter)) { - if (i == startPos) { - setErrorState(ParseErrorState.EMPTY_COLUMN); - return -1; - } - break; - } - i++; + final int endPos = nextStringEndPos(bytes, startPos, limit, delimiter); + if (endPos < 0) { + return -1; } - if (i > startPos && - (Character.isWhitespace(bytes[startPos]) || Character.isWhitespace(bytes[(i - 1)]))) { + if (endPos > startPos && + (Character.isWhitespace(bytes[startPos]) || Character.isWhitespace(bytes[(endPos - 1)]))) { setErrorState(ParseErrorState.NUMERIC_VALUE_ILLEGAL_CHARACTER); return -1; } - String str = new String(bytes, startPos, i - startPos); + String str = new String(bytes, startPos, endPos - startPos); try { this.result = new BigInteger(str); - return (i == limit) ? limit : i + delimiter.length; + return (endPos == limit) ? limit : endPos + delimiter.length; } catch (NumberFormatException e) { setErrorState(ParseErrorState.NUMERIC_VALUE_FORMAT_ERROR); return -1; @@ -84,7 +74,7 @@ public BigInteger getLastResult() { * @param startPos The offset to start the parsing. * @param length The length of the byte sequence (counting from the offset). * @return The parsed value. - * @throws NumberFormatException Thrown when the value cannot be parsed because the text + * @throws IllegalArgumentException Thrown when the value cannot be parsed because the text * represents not a correct number. */ public static final BigInteger parseField(byte[] bytes, int startPos, int length) { @@ -101,26 +91,18 @@ public static final BigInteger parseField(byte[] bytes, int startPos, int length * @param length The length of the byte sequence (counting from the offset). * @param delimiter The delimiter that terminates the field. * @return The parsed value. - * @throws NumberFormatException Thrown when the value cannot be parsed because the text + * @throws IllegalArgumentException Thrown when the value cannot be parsed because the text * represents not a correct number. */ public static final BigInteger parseField(byte[] bytes, int startPos, int length, char delimiter) { - if (length <= 0) { - throw new NumberFormatException("Invalid input: Empty string"); - } - int i = 0; - final byte delByte = (byte) delimiter; - - while (i < length && bytes[startPos + i] != delByte) { - i++; - } + final int limitedLen = nextStringLength(bytes, startPos, length, delimiter); - if (i > 0 && - (Character.isWhitespace(bytes[startPos]) || Character.isWhitespace(bytes[startPos + i - 1]))) { + if (limitedLen > 0 && + (Character.isWhitespace(bytes[startPos]) || Character.isWhitespace(bytes[startPos + limitedLen - 1]))) { throw new NumberFormatException("There is leading or trailing whitespace in the numeric field."); } - String str = new String(bytes, startPos, i); + final String str = new String(bytes, startPos, limitedLen); return new BigInteger(str); } } diff --git a/flink-core/src/main/java/org/apache/flink/types/parser/DoubleParser.java b/flink-core/src/main/java/org/apache/flink/types/parser/DoubleParser.java index 8af496dfbd50e..2474adf6e40d8 100644 --- a/flink-core/src/main/java/org/apache/flink/types/parser/DoubleParser.java +++ b/flink-core/src/main/java/org/apache/flink/types/parser/DoubleParser.java @@ -33,31 +33,21 @@ public class DoubleParser extends FieldParser { @Override public int parseField(byte[] bytes, int startPos, int limit, byte[] delimiter, Double reusable) { - int i = startPos; - - final int delimLimit = limit - delimiter.length + 1; - - while (i < limit) { - if (i < delimLimit && delimiterNext(bytes, i, delimiter)) { - if (i == startPos) { - setErrorState(ParseErrorState.EMPTY_COLUMN); - return -1; - } - break; - } - i++; + final int endPos = nextStringEndPos(bytes, startPos, limit, delimiter); + if (endPos < 0) { + return -1; } - if (i > startPos && - (Character.isWhitespace(bytes[startPos]) || Character.isWhitespace(bytes[(i - 1)]))) { + if (endPos > startPos && + (Character.isWhitespace(bytes[startPos]) || Character.isWhitespace(bytes[(endPos - 1)]))) { setErrorState(ParseErrorState.NUMERIC_VALUE_ILLEGAL_CHARACTER); return -1; } - String str = new String(bytes, startPos, i - startPos); + String str = new String(bytes, startPos, endPos - startPos); try { this.result = Double.parseDouble(str); - return (i == limit) ? limit : i + delimiter.length; + return (endPos == limit) ? limit : endPos + delimiter.length; } catch (NumberFormatException e) { setErrorState(ParseErrorState.NUMERIC_VALUE_FORMAT_ERROR); return -1; @@ -83,7 +73,7 @@ public Double getLastResult() { * @param startPos The offset to start the parsing. * @param length The length of the byte sequence (counting from the offset). * @return The parsed value. - * @throws NumberFormatException Thrown when the value cannot be parsed because the text + * @throws IllegalArgumentException Thrown when the value cannot be parsed because the text * represents not a correct number. */ public static final double parseField(byte[] bytes, int startPos, int length) { @@ -100,26 +90,18 @@ public static final double parseField(byte[] bytes, int startPos, int length) { * @param length The length of the byte sequence (counting from the offset). * @param delimiter The delimiter that terminates the field. * @return The parsed value. - * @throws NumberFormatException Thrown when the value cannot be parsed because the text + * @throws IllegalArgumentException Thrown when the value cannot be parsed because the text * represents not a correct number. */ public static final double parseField(byte[] bytes, int startPos, int length, char delimiter) { - if (length <= 0) { - throw new NumberFormatException("Invalid input: Empty string"); - } - int i = 0; - final byte delByte = (byte) delimiter; - - while (i < length && bytes[startPos + i] != delByte) { - i++; - } + final int limitedLen = nextStringLength(bytes, startPos, length, delimiter); - if (i > 0 && - (Character.isWhitespace(bytes[startPos]) || Character.isWhitespace(bytes[startPos + i - 1]))) { + if (limitedLen > 0 && + (Character.isWhitespace(bytes[startPos]) || Character.isWhitespace(bytes[startPos + limitedLen - 1]))) { throw new NumberFormatException("There is leading or trailing whitespace in the numeric field."); } - String str = new String(bytes, startPos, i); + final String str = new String(bytes, startPos, limitedLen); return Double.parseDouble(str); } } diff --git a/flink-core/src/main/java/org/apache/flink/types/parser/DoubleValueParser.java b/flink-core/src/main/java/org/apache/flink/types/parser/DoubleValueParser.java index 5c657be779484..10b43c37bbb57 100644 --- a/flink-core/src/main/java/org/apache/flink/types/parser/DoubleValueParser.java +++ b/flink-core/src/main/java/org/apache/flink/types/parser/DoubleValueParser.java @@ -32,34 +32,23 @@ public class DoubleValueParser extends FieldParser { @Override public int parseField(byte[] bytes, int startPos, int limit, byte[] delimiter, DoubleValue reusable) { - - int i = startPos; - - final int delimLimit = limit - delimiter.length + 1; - - while (i < limit) { - if (i < delimLimit && delimiterNext(bytes, i, delimiter)) { - if (i == startPos) { - setErrorState(ParseErrorState.EMPTY_COLUMN); - return -1; - } - break; - } - i++; + final int endPos = nextStringEndPos(bytes, startPos, limit, delimiter); + if (endPos < 0) { + return -1; } - - if (i > startPos && - (Character.isWhitespace(bytes[startPos]) || Character.isWhitespace(bytes[i - 1]))) { + + if (endPos > startPos && + (Character.isWhitespace(bytes[startPos]) || Character.isWhitespace(bytes[(endPos - 1)]))) { setErrorState(ParseErrorState.NUMERIC_VALUE_ILLEGAL_CHARACTER); return -1; } - String str = new String(bytes, startPos, i - startPos); + String str = new String(bytes, startPos, endPos - startPos); try { double value = Double.parseDouble(str); reusable.setValue(value); this.result = reusable; - return (i == limit) ? limit : i + delimiter.length; + return (endPos == limit) ? limit : endPos + delimiter.length; } catch (NumberFormatException e) { setErrorState(ParseErrorState.NUMERIC_VALUE_FORMAT_ERROR); diff --git a/flink-core/src/main/java/org/apache/flink/types/parser/FieldParser.java b/flink-core/src/main/java/org/apache/flink/types/parser/FieldParser.java index a1b9c5f4fda66..200d239dc77a6 100644 --- a/flink-core/src/main/java/org/apache/flink/types/parser/FieldParser.java +++ b/flink-core/src/main/java/org/apache/flink/types/parser/FieldParser.java @@ -174,6 +174,49 @@ protected void setErrorState(ParseErrorState error) { public ParseErrorState getErrorState() { return this.errorState; } + + /** + * Returns the end position of a string. Sets the error state if the column is empty. + * + * @return the end position of the string or -1 if an error occurred + */ + protected final int nextStringEndPos(byte[] bytes, int startPos, int limit, byte[] delimiter) { + int endPos = startPos; + + final int delimLimit = limit - delimiter.length + 1; + + while (endPos < limit) { + if (endPos < delimLimit && delimiterNext(bytes, endPos, delimiter)) { + if (endPos == startPos) { + setErrorState(ParseErrorState.EMPTY_COLUMN); + return -1; + } + break; + } + endPos++; + } + + return endPos; + } + + /** + * Returns the length of a string. Throws an exception if the column is empty. + * + * @return the length of the string + */ + protected static final int nextStringLength(byte[] bytes, int startPos, int length, char delimiter) { + if (length <= 0) { + throw new IllegalArgumentException("Invalid input: Empty string"); + } + int limitedLength = 0; + final byte delByte = (byte) delimiter; + + while (limitedLength < length && bytes[startPos + limitedLength] != delByte) { + limitedLength++; + } + + return limitedLength; + } // -------------------------------------------------------------------------------------------- // Mapping from types to parsers @@ -222,5 +265,10 @@ public static Class> getParserForType(Class type) { PARSERS.put(FloatValue.class, FloatValueParser.class); PARSERS.put(DoubleValue.class, DoubleValueParser.class); PARSERS.put(BooleanValue.class, BooleanValueParser.class); + + // SQL date/time types + PARSERS.put(java.sql.Time.class, SqlTimeParser.class); + PARSERS.put(java.sql.Date.class, SqlDateParser.class); + PARSERS.put(java.sql.Timestamp.class, SqlTimestampParser.class); } } diff --git a/flink-core/src/main/java/org/apache/flink/types/parser/FloatParser.java b/flink-core/src/main/java/org/apache/flink/types/parser/FloatParser.java index 3304f24d591db..e76484e60b76a 100644 --- a/flink-core/src/main/java/org/apache/flink/types/parser/FloatParser.java +++ b/flink-core/src/main/java/org/apache/flink/types/parser/FloatParser.java @@ -30,34 +30,22 @@ public class FloatParser extends FieldParser { private float result; @Override - public int parseField(byte[] bytes, int startPos, int limit, byte[] delimiter, Float - reusable) { - - int i = startPos; - - final int delimLimit = limit - delimiter.length + 1; - - while (i < limit) { - if (i < delimLimit && delimiterNext(bytes, i, delimiter)) { - if (i == startPos) { - setErrorState(ParseErrorState.EMPTY_COLUMN); - return -1; - } - break; - } - i++; + public int parseField(byte[] bytes, int startPos, int limit, byte[] delimiter, Float reusable) { + final int endPos = nextStringEndPos(bytes, startPos, limit, delimiter); + if (endPos < 0) { + return -1; } - if (i > startPos && - (Character.isWhitespace(bytes[startPos]) || Character.isWhitespace(bytes[i - 1]))) { + if (endPos > startPos && + (Character.isWhitespace(bytes[startPos]) || Character.isWhitespace(bytes[endPos - 1]))) { setErrorState(ParseErrorState.NUMERIC_VALUE_ILLEGAL_CHARACTER); return -1; } - String str = new String(bytes, startPos, i - startPos); + String str = new String(bytes, startPos, endPos - startPos); try { this.result = Float.parseFloat(str); - return (i == limit) ? limit : i + delimiter.length; + return (endPos == limit) ? limit : endPos + delimiter.length; } catch (NumberFormatException e) { setErrorState(ParseErrorState.NUMERIC_VALUE_FORMAT_ERROR); return -1; @@ -83,7 +71,7 @@ public Float getLastResult() { * @param startPos The offset to start the parsing. * @param length The length of the byte sequence (counting from the offset). * @return The parsed value. - * @throws NumberFormatException Thrown when the value cannot be parsed because the text + * @throws IllegalArgumentException Thrown when the value cannot be parsed because the text * represents not a correct number. */ public static final float parseField(byte[] bytes, int startPos, int length) { @@ -100,26 +88,18 @@ public static final float parseField(byte[] bytes, int startPos, int length) { * @param length The length of the byte sequence (counting from the offset). * @param delimiter The delimiter that terminates the field. * @return The parsed value. - * @throws NumberFormatException Thrown when the value cannot be parsed because the text + * @throws IllegalArgumentException Thrown when the value cannot be parsed because the text * represents not a correct number. */ public static final float parseField(byte[] bytes, int startPos, int length, char delimiter) { - if (length <= 0) { - throw new NumberFormatException("Invalid input: Empty string"); - } - int i = 0; - final byte delByte = (byte) delimiter; - - while (i < length && bytes[startPos + i] != delByte) { - i++; - } + final int limitedLen = nextStringLength(bytes, startPos, length, delimiter); - if (i > 0 && - (Character.isWhitespace(bytes[startPos]) || Character.isWhitespace(bytes[startPos + i - 1]))) { + if (limitedLen > 0 && + (Character.isWhitespace(bytes[startPos]) || Character.isWhitespace(bytes[startPos + limitedLen - 1]))) { throw new NumberFormatException("There is leading or trailing whitespace in the numeric field."); } - String str = new String(bytes, startPos, i); + final String str = new String(bytes, startPos, limitedLen); return Float.parseFloat(str); } } diff --git a/flink-core/src/main/java/org/apache/flink/types/parser/FloatValueParser.java b/flink-core/src/main/java/org/apache/flink/types/parser/FloatValueParser.java index 26ee47bae05f4..a834f22a4cd48 100644 --- a/flink-core/src/main/java/org/apache/flink/types/parser/FloatValueParser.java +++ b/flink-core/src/main/java/org/apache/flink/types/parser/FloatValueParser.java @@ -32,34 +32,23 @@ public class FloatValueParser extends FieldParser { @Override public int parseField(byte[] bytes, int startPos, int limit, byte[] delimiter, FloatValue reusable) { - - int i = startPos; - - final int delimLimit = limit - delimiter.length + 1; - - while (i < limit) { - if (i < delimLimit && delimiterNext(bytes, i, delimiter)) { - if (i == startPos) { - setErrorState(ParseErrorState.EMPTY_COLUMN); - return -1; - } - break; - } - i++; + final int endPos = nextStringEndPos(bytes, startPos, limit, delimiter); + if (endPos < 0) { + return -1; } - - if (i > startPos && - (Character.isWhitespace(bytes[startPos]) || Character.isWhitespace(bytes[i - 1]))) { + + if (endPos > startPos && + (Character.isWhitespace(bytes[startPos]) || Character.isWhitespace(bytes[endPos - 1]))) { setErrorState(ParseErrorState.NUMERIC_VALUE_ILLEGAL_CHARACTER); return -1; } - String str = new String(bytes, startPos, i - startPos); + String str = new String(bytes, startPos, endPos - startPos); try { float value = Float.parseFloat(str); reusable.setValue(value); this.result = reusable; - return (i == limit) ? limit : i + delimiter.length; + return (endPos == limit) ? limit : endPos + delimiter.length; } catch (NumberFormatException e) { setErrorState(ParseErrorState.NUMERIC_VALUE_FORMAT_ERROR); diff --git a/flink-core/src/main/java/org/apache/flink/types/parser/SqlDateParser.java b/flink-core/src/main/java/org/apache/flink/types/parser/SqlDateParser.java new file mode 100644 index 0000000000000..859dcf8809360 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/types/parser/SqlDateParser.java @@ -0,0 +1,108 @@ +/* + * 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.types.parser; + +import java.sql.Date; +import org.apache.flink.annotation.PublicEvolving; + +/** + * Parses a text field into a {@link java.sql.Date}. + */ +@PublicEvolving +public class SqlDateParser extends FieldParser { + + private static final Date DATE_INSTANCE = new Date(0L); + + private Date result; + + @Override + public int parseField(byte[] bytes, int startPos, int limit, byte[] delimiter, Date reusable) { + final int endPos = nextStringEndPos(bytes, startPos, limit, delimiter); + if (endPos < 0) { + return -1; + } + + if (endPos > startPos && + (Character.isWhitespace(bytes[startPos]) || Character.isWhitespace(bytes[(endPos - 1)]))) { + setErrorState(ParseErrorState.NUMERIC_VALUE_ILLEGAL_CHARACTER); + return -1; + } + + String str = new String(bytes, startPos, endPos - startPos); + try { + this.result = Date.valueOf(str); + return (endPos == limit) ? limit : endPos + delimiter.length; + } catch (IllegalArgumentException e) { + setErrorState(ParseErrorState.NUMERIC_VALUE_FORMAT_ERROR); + return -1; + } + } + + @Override + public Date createValue() { + return DATE_INSTANCE; + } + + @Override + public Date getLastResult() { + return this.result; + } + + /** + * Static utility to parse a field of type Date from a byte sequence that represents text + * characters + * (such as when read from a file stream). + * + * @param bytes The bytes containing the text data that should be parsed. + * @param startPos The offset to start the parsing. + * @param length The length of the byte sequence (counting from the offset). + * @return The parsed value. + * @throws IllegalArgumentException Thrown when the value cannot be parsed because the text + * represents not a correct number. + */ + public static final Date parseField(byte[] bytes, int startPos, int length) { + return parseField(bytes, startPos, length, (char) 0xffff); + } + + /** + * Static utility to parse a field of type Date from a byte sequence that represents text + * characters + * (such as when read from a file stream). + * + * @param bytes The bytes containing the text data that should be parsed. + * @param startPos The offset to start the parsing. + * @param length The length of the byte sequence (counting from the offset). + * @param delimiter The delimiter that terminates the field. + * @return The parsed value. + * @throws IllegalArgumentException Thrown when the value cannot be parsed because the text + * represents not a correct number. + */ + public static final Date parseField(byte[] bytes, int startPos, int length, char delimiter) { + final int limitedLen = nextStringLength(bytes, startPos, length, delimiter); + + if (limitedLen > 0 && + (Character.isWhitespace(bytes[startPos]) || Character.isWhitespace(bytes[startPos + limitedLen - 1]))) { + throw new NumberFormatException("There is leading or trailing whitespace in the numeric field."); + } + + final String str = new String(bytes, startPos, limitedLen); + return Date.valueOf(str); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/types/parser/SqlTimeParser.java b/flink-core/src/main/java/org/apache/flink/types/parser/SqlTimeParser.java new file mode 100644 index 0000000000000..fbddadc3a0dee --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/types/parser/SqlTimeParser.java @@ -0,0 +1,102 @@ +/* + * 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.types.parser; + +import java.sql.Time; +import org.apache.flink.annotation.PublicEvolving; + +/** + * Parses a text field into a {@link Time}. + */ +@PublicEvolving +public class SqlTimeParser extends FieldParser