From eeb1fc284d6879404b05eab422ed898a9440cdb5 Mon Sep 17 00:00:00 2001 From: yuzhongliu Date: Mon, 29 Aug 2016 19:03:15 +0800 Subject: [PATCH 1/9] #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 6e316e7e75044..8d296192eea75 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 2ebb917f2f1ab..e801777de36fe 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 055743ad65c6213e8fe8d422e929e795c3bad8f1 Mon Sep 17 00:00:00 2001 From: yuzhongliu Date: Tue, 30 Aug 2016 14:05:08 +0800 Subject: [PATCH 2/9] #4450 update storm version to 1.0.0 in flink-storm-examples --- 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 | 6 +- .../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 | 14 +++-- 50 files changed, 229 insertions(+), 224 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 b43b24d077985..9f70befcc0032 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,9 @@ */ 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; 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 5e723db2e8a36..c70b8219850bf 100644 --- a/pom.xml +++ b/pom.xml @@ -53,8 +53,8 @@ under the License. tools/force-shading flink-annotations - flink-shaded-hadoop - flink-shaded-curator + flink-core flink-java flink-scala @@ -63,18 +63,20 @@ under the License. flink-optimizer flink-streaming-java flink-streaming-scala - flink-streaming-connectors + flink-clients - flink-tests + flink-contrib flink-dist - flink-metrics + From 409827d8e7ac69743b9c7f98774c001f6551457c Mon Sep 17 00:00:00 2001 From: yuzhongliu Date: Tue, 30 Aug 2016 14:10:47 +0800 Subject: [PATCH 3/9] #4450 reverse for uncorrect change submit pom.xml --- flink-contrib/pom.xml | 4 ++-- pom.xml | 14 ++++++-------- 2 files changed, 8 insertions(+), 10 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 c70b8219850bf..1ab2a759ab2a4 100644 --- a/pom.xml +++ b/pom.xml @@ -53,8 +53,8 @@ under the License. tools/force-shading flink-annotations - + flink-shaded-hadoop + flink-shaded-curator flink-core flink-java flink-scala @@ -63,20 +63,18 @@ under the License. flink-optimizer flink-streaming-java flink-streaming-scala - flink-clients - + flink-quickstart flink-contrib flink-dist - + flink-metrics From eebd8c534933fb2639dc248469ba56a2ed053cb7 Mon Sep 17 00:00:00 2001 From: yuzhongliu Date: Tue, 30 Aug 2016 14:12:00 +0800 Subject: [PATCH 4/9] #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 1ab2a759ab2a4..5e723db2e8a36 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 89d04437c74567fe21949f0b3ac5730e42002db2 Mon Sep 17 00:00:00 2001 From: yuzhongliu Date: Wed, 31 Aug 2016 11:12:14 +0800 Subject: [PATCH 5/9] #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 3a20d1957a986daa0eb4758d701cd9cd0ac98839 Mon Sep 17 00:00:00 2001 From: yuzhongliu Date: Wed, 31 Aug 2016 11:14:21 +0800 Subject: [PATCH 6/9] #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 33bd4f377074cd680299891a97734cd8658b4129 Mon Sep 17 00:00:00 2001 From: yuzhongliu Date: Thu, 1 Sep 2016 12:58:56 +0800 Subject: [PATCH 7/9] #4450 sync to https://github.com/apache/flink.git --- .travis.yml | 14 +- LICENSE | 5 +- docs/README.md | 42 +- docs/_config.yml | 17 +- docs/_includes/navbar.html | 117 - docs/_includes/sidenav.html | 149 ++ docs/_layouts/base.html | 33 +- docs/_layouts/plain.html | 130 +- .../redirect.html} | 11 +- docs/apis/batch/fig/LICENSE.txt | 17 - docs/apis/batch/libs/gelly.md | 26 - docs/apis/batch/libs/table.md | 26 - docs/apis/streaming/fig/LICENSE.txt | 17 - docs/apis/streaming_guide.md | 26 - docs/concepts/{concepts.md => index.md} | 33 +- .../common/index.md => dev/api_concepts.md} | 33 +- docs/{apis/index.md => dev/apis.md} | 5 +- docs/{apis => dev}/batch/connectors.md | 18 +- .../batch/dataset_transformations.md | 9 +- docs/{apis => dev}/batch/examples.md | 8 +- docs/{apis => dev}/batch/fault_tolerance.md | 6 +- .../batch/hadoop_compatibility.md | 7 +- docs/{apis => dev}/batch/index.md | 53 +- docs/{apis => dev}/batch/iterations.md | 3 +- docs/{apis => dev}/batch/python.md | 15 +- .../{apis => dev}/batch/zip_elements_guide.md | 8 +- docs/{apis => dev}/cluster_execution.md | 9 +- .../streaming => dev}/connectors/cassandra.md | 13 +- .../connectors/elasticsearch.md | 11 +- .../connectors/elasticsearch2.md | 11 +- .../connectors/filesystem_sink.md | 64 +- .../streaming => dev}/connectors/index.md | 13 +- .../streaming => dev}/connectors/kafka.md | 48 +- .../streaming => dev}/connectors/kinesis.md | 33 +- .../streaming => dev}/connectors/nifi.md | 29 +- .../streaming => dev}/connectors/rabbitmq.md | 11 +- .../streaming => dev}/connectors/redis.md | 15 +- .../streaming => dev}/connectors/twitter.md | 28 +- .../index.md => dev/datastream_api.md} | 64 +- docs/{apis/streaming => dev}/event_time.md | 22 +- .../event_timestamp_extractors.md | 40 +- .../event_timestamps_watermarks.md | 13 +- docs/{apis/batch/libs => dev}/index.md | 14 +- docs/{apis => dev}/java8.md | 10 +- .../libs/index.md => dev/libraries.md} | 11 +- docs/{apis/streaming => dev}/libs/cep.md | 17 +- .../libs/gelly/graph_algorithms.md | 7 +- .../batch => dev}/libs/gelly/graph_api.md | 17 +- .../libs/gelly/graph_generators.md | 7 +- docs/{apis/batch => dev}/libs/gelly/index.md | 17 +- .../libs/gelly/iterative_graph_processing.md | 19 +- .../libs/gelly/library_methods.md | 7 +- docs/{apis/batch => dev}/libs/ml/als.md | 23 +- .../libs/ml/contribution_guide.md | 6 +- .../batch => dev}/libs/ml/cross_validation.md | 8 +- .../batch => dev}/libs/ml/distance_metrics.md | 8 +- docs/{apis/batch => dev}/libs/ml/index.md | 23 +- docs/{apis/batch => dev}/libs/ml/knn.md | 13 +- .../batch => dev}/libs/ml/min_max_scaler.md | 6 +- .../libs/ml/multiple_linear_regression.md | 18 +- .../batch => dev}/libs/ml/optimization.md | 11 +- docs/{apis/batch => dev}/libs/ml/pipelines.md | 58 +- .../libs/ml/polynomial_features.md | 11 +- .../{apis/batch => dev}/libs/ml/quickstart.md | 19 +- .../batch => dev}/libs/ml/standard_scaler.md | 27 +- docs/{apis/batch => dev}/libs/ml/svm.md | 9 +- .../libs}/storm_compatibility.md | 14 +- docs/{apis => dev}/local_execution.md | 9 +- .../fig/LICENSE.txt => dev/quickstarts.md} | 13 +- docs/{apis => dev}/scala_api_extensions.md | 15 +- docs/{apis => dev}/scala_shell.md | 12 +- docs/{apis/streaming => dev}/state.md | 10 +- .../{apis/streaming => dev}/state_backends.md | 13 +- docs/{apis/table.md => dev/table_api.md} | 104 +- .../{internals => dev}/types_serialization.md | 25 +- docs/{apis/streaming => dev}/windows.md | 189 +- docs/{internals => }/fig/ClientJmTm.svg | 0 docs/{setup => }/fig/FlinkOnYarn.svg | 0 .../fig/back_pressure_sampling.png | Bin .../fig/back_pressure_sampling_high.png | Bin .../back_pressure_sampling_in_progress.png | Bin .../fig/back_pressure_sampling_ok.png | Bin docs/{internals => }/fig/checkpointing.svg | 0 docs/{concepts => }/fig/checkpoints.svg | 0 .../fig/event_ingestion_processing_time.svg | 0 docs/{setup => }/fig/flink-on-emr.png | Bin .../libs => }/fig/gelly-example-graph.png | Bin .../batch/libs => }/fig/gelly-filter.png | Bin .../batch/libs => }/fig/gelly-gsa-sssp1.png | Bin .../libs => }/fig/gelly-reduceOnEdges.png | Bin .../libs => }/fig/gelly-reduceOnNeighbors.png | Bin .../{apis/batch/libs => }/fig/gelly-union.png | Bin .../batch/libs => }/fig/gelly-vc-sssp1.png | Bin .../fig/iterations_delta_iterate_operator.png | Bin ...rations_delta_iterate_operator_example.png | Bin .../fig/iterations_iterate_operator.png | Bin .../iterations_iterate_operator_example.png | Bin .../batch => }/fig/iterations_supersteps.png | Bin .../fig/job_and_execution_graph.svg | 0 docs/{internals => }/fig/job_status.svg | 0 .../fig/jobmanager_ha_overview.png | Bin docs/{apis/streaming => fig}/non-windowed.svg | 0 docs/{concepts => }/fig/parallel_dataflow.svg | 0 .../fig/parallel_streams_watermarks.svg | 0 .../{apis/common => }/fig/plan_visualizer.png | Bin docs/{concepts => }/fig/processes.svg | 0 docs/{concepts => }/fig/program_dataflow.svg | 0 .../fig/projects_dependencies.svg | 0 docs/{apis/streaming => }/fig/rescale.svg | 0 .../fig/savepoints-overview.png | Bin .../fig/savepoints-program_ids.png | Bin .../streaming => fig}/session-windows.svg | 0 .../streaming => fig}/sliding-windows.svg | 0 docs/{concepts => }/fig/slot_sharing.svg | 0 docs/{internals => }/fig/slots.svg | 0 docs/{setup => }/fig/slots_parallelism.svg | 0 docs/{internals => }/fig/stack.svg | 0 docs/{internals => }/fig/state_machine.svg | 0 .../{concepts => }/fig/state_partitioning.svg | 0 docs/{internals => }/fig/stream_aligning.svg | 0 docs/{internals => }/fig/stream_barriers.svg | 0 .../fig/stream_watermark_in_order.svg | 0 .../fig/stream_watermark_out_of_order.svg | 0 docs/{concepts => }/fig/tasks_chains.svg | 0 docs/{concepts => }/fig/tasks_slots.svg | 0 .../{apis/streaming => }/fig/times_clocks.svg | 0 .../streaming => fig}/tumbling-windows.svg | 0 .../fig/vertex-centric supersteps.png | Bin docs/{concepts => }/fig/windows.svg | 0 docs/index.md | 27 +- docs/internals/_draft_distributed_akka.md | 47 - docs/internals/add_operator.md | 24 +- docs/internals/coding_guidelines.md | 25 - docs/internals/fig/LICENSE.txt | 17 - docs/internals/general_arch.md | 32 +- docs/internals/how_to_contribute.md | 25 - docs/internals/ide_setup.md | 19 +- docs/internals/index.md | 4 + docs/internals/job_scheduling.md | 23 +- docs/internals/stream_checkpointing.md | 25 +- docs/libs/cep/index.md | 25 - docs/libs/gelly_guide.md | 25 - docs/libs/index.md | 25 - docs/libs/ml/als.md | 25 - docs/libs/ml/contribution_guide.md | 25 - docs/libs/ml/distance_metrics.md | 25 - docs/libs/ml/index.md | 25 - docs/libs/ml/min_max_scaler.md | 25 - docs/libs/ml/multiple_linear_regression.md | 25 - docs/libs/ml/optimization.md | 25 - docs/libs/ml/pipelines.md | 25 - docs/libs/ml/polynomial_features.md | 25 - docs/libs/ml/quickstart.md | 25 - docs/libs/ml/standard_scaler.md | 25 - docs/libs/ml/svm.md | 25 - docs/libs/table.md | 29 - .../back_pressure.md} | 14 +- docs/{apis => monitoring}/best_practices.md | 7 +- docs/monitoring/index.md | 25 + docs/{internals => monitoring}/logging.md | 16 +- docs/{apis => monitoring}/metrics.md | 68 +- .../rest_api.md} | 15 +- docs/page/css/flink.css | 174 +- docs/page/font-awesome/css/font-awesome.css | 2199 +++++++++++++++++ .../font-awesome/css/font-awesome.min.css | 4 + docs/page/font-awesome/fonts/FontAwesome.otf | Bin 0 -> 124988 bytes .../fonts/fontawesome-webfont.eot | Bin 0 -> 76518 bytes .../fonts/fontawesome-webfont.svg | 685 +++++ .../fonts/fontawesome-webfont.ttf | Bin 0 -> 152796 bytes .../fonts/fontawesome-webfont.woff | Bin 0 -> 90412 bytes .../fonts/fontawesome-webfont.woff2 | Bin 0 -> 71896 bytes docs/quickstart/java_api_quickstart.md | 22 +- docs/quickstart/run_example_quickstart.md | 7 +- docs/quickstart/scala_api_quickstart.md | 21 +- docs/quickstart/setup_quickstart.md | 11 +- docs/redirects/back_pressure.md | 24 + docs/redirects/basic_api_concepts.md | 24 + docs/redirects/best_practices.md | 24 + docs/redirects/cassandra.md | 24 + docs/redirects/cep.md | 24 + docs/redirects/cli.md | 24 + docs/redirects/cluster_execution.md | 24 + docs/redirects/concepts.md | 24 + docs/{apis => redirects}/connectors.md | 7 +- docs/redirects/datastream_api.md | 24 + docs/redirects/elasticsearch.md | 24 + docs/redirects/elasticsearch2.md | 24 + docs/redirects/event_time.md | 24 + docs/redirects/event_timestamp_extractors.md | 24 + docs/redirects/event_timestamps_watermarks.md | 24 + docs/redirects/fault_tolerance.md | 24 + docs/redirects/filesystem_sink.md | 24 + docs/redirects/gelly.md | 24 + docs/redirects/java8.md | 24 + docs/redirects/kafka.md | 24 + docs/redirects/kinesis.md | 24 + docs/redirects/local_execution.md | 24 + docs/redirects/metrics.md | 24 + docs/redirects/ml.md | 24 + docs/redirects/programming_guide.md | 24 + docs/redirects/rest_api.md | 24 + docs/redirects/savepoints.md | 24 + docs/redirects/scala_api_extensions.md | 24 + docs/redirects/scala_shell.md | 24 + docs/redirects/state.md | 24 + docs/redirects/state_backends.md | 24 + docs/redirects/storm_compat.md | 24 + docs/redirects/streaming_guide.md | 24 + docs/redirects/table.md | 24 + docs/redirects/types_serialization.md | 24 + docs/redirects/windows.md | 24 + docs/setup/aws.md | 12 +- docs/setup/building.md | 5 +- docs/{apis => setup}/cli.md | 8 +- docs/setup/cluster_setup.md | 7 +- docs/setup/config.md | 46 +- docs/setup/deployment.md | 24 + .../streaming => setup}/fault_tolerance.md | 22 +- docs/setup/fig/LICENSE.txt | 17 - docs/setup/gce_setup.md | 6 +- docs/setup/index.md | 6 +- docs/setup/jobmanager_high_availability.md | 53 +- docs/setup/local_setup.md | 7 +- docs/{apis/streaming => setup}/savepoints.md | 15 +- docs/setup/yarn_setup.md | 12 +- .../org/apache/flink/client/CliFrontend.java | 7 +- .../apache/flink/client/cli/DefaultCLI.java | 2 +- .../flink/client/program/ClusterClient.java | 84 +- .../program/StandaloneClusterClient.java | 7 +- .../client/CliFrontendSavepointTest.java | 12 +- .../src/test/resources/log4j-test.properties | 2 +- .../flink-statebackend-rocksdb/pom.xml | 7 + .../streaming/state/AbstractRocksDBState.java | 113 +- .../streaming/state/RocksDBFoldingState.java | 25 +- .../state/RocksDBKeyedStateBackend.java | 986 ++++++++ .../streaming/state/RocksDBListState.java | 24 +- .../streaming/state/RocksDBReducingState.java | 26 +- .../streaming/state/RocksDBStateBackend.java | 934 +------ .../streaming/state/RocksDBValueState.java | 27 +- .../FullyAsyncRocksDBStateBackendTest.java | 65 - ...est.java => RocksDBAsyncSnapshotTest.java} | 225 +- .../state/RocksDBMergeIteratorTest.java | 140 ++ .../state/RocksDBStateBackendConfigTest.java | 209 +- .../state/RocksDBStateBackendTest.java | 29 +- .../src/test/resources/log4j-test.properties | 2 +- .../src/test/resources/log4j.properties | 2 +- .../tests/StormFieldsGroupingITCase.java | 34 +- .../flink/storm/wrappers/BoltWrapper.java | 2 +- .../flink/storm/wrappers/BoltWrapperTest.java | 4 +- flink-core/pom.xml | 1 - .../flink/api/common/ExecutionConfig.java | 51 +- .../flink/api/common/JobExecutionResult.java | 4 +- .../org/apache/flink/api/common/TaskInfo.java | 13 +- .../flink/api/common/io/FileInputFormat.java | 20 +- .../flink/api/common/io}/FilePathFilter.java | 9 +- .../api/common/io/GlobFilePathFilter.java | 111 + .../common/operators/CollectionExecutor.java | 8 +- .../operators/base/ReduceOperatorBase.java | 7 +- .../api/common/state/StateDescriptor.java | 1 + .../base/array/StringArraySerializer.java | 2 +- .../flink/configuration/ConfigConstants.java | 187 +- .../flink/core/fs/FSDataOutputStream.java | 2 + .../java/org/apache/flink/core/fs/Path.java | 10 +- .../core/fs/local/LocalDataOutputStream.java | 5 + .../flink/core/fs/local/LocalFileStatus.java | 8 + .../memory/ByteArrayOutputStreamWithPos.java | 281 +++ .../apache/flink/util/ConfigurationUtil.java | 101 + .../apache/flink/util/InstantiationUtil.java | 34 +- .../java/org/apache/flink/util/MathUtils.java | 15 + .../functions/util/RuntimeUDFContextTest.java | 2 +- .../api/common/io/DefaultFilterTest.java | 70 + .../api/common/io/FileInputFormatTest.java | 174 +- .../api/common/io/GlobFilePathFilterTest.java | 141 ++ .../api/common/io/RichInputFormatTest.java | 2 +- .../api/common/io/RichOutputFormatTest.java | 2 +- .../operators/GenericDataSinkBaseTest.java | 2 +- .../operators/GenericDataSourceBaseTest.java | 2 +- .../base/FlatMapOperatorCollectionTest.java | 2 +- .../base/InnerJoinOperatorBaseTest.java | 2 +- .../operators/base/MapOperatorTest.java | 2 +- .../base/PartitionMapOperatorTest.java | 2 +- .../base/array/StringArraySerializerTest.java | 9 + .../flink/util/ConfigurationUtilTest.java | 115 + .../org/apache/flink/util/MathUtilTest.java | 31 + flink-dist/pom.xml | 7 +- flink-dist/src/main/flink-bin/bin/config.sh | 21 +- .../src/main/flink-bin/bin/start-cluster.sh | 2 +- .../src/main/flink-bin/bin/stop-cluster.sh | 2 +- flink-dist/src/main/resources/flink-conf.yaml | 16 +- ...ontinuousFileMonitoringFunctionITCase.java | 4 +- .../ContinuousFileMonitoringTest.java | 13 +- .../flink/hdfstests/FileStateBackendTest.java | 161 +- .../api/java/operators/DistinctOperator.java | 82 +- .../api/java/operators/ReduceOperator.java | 2 +- .../base/CoGroupOperatorCollectionTest.java | 2 +- .../base/GroupReduceOperatorTest.java | 2 +- .../base/InnerJoinOperatorBaseTest.java | 2 +- .../operators/base/ReduceOperatorTest.java | 2 +- .../translation/DistinctTranslationTest.java | 52 +- .../java/org/apache/flink/cep/nfa/NFA.java | 2 +- .../operator/AbstractCEPPatternOperator.java | 44 +- .../AbstractKeyedCEPPatternOperator.java | 27 +- .../flink/cep/operator/CEPOperatorTest.java | 94 +- .../flink/graph/driver/GraphMetrics.java | 232 ++ .../graph/examples/ClusteringCoefficient.java | 14 +- .../org/apache/flink/graph/examples/HITS.java | 2 +- .../flink/graph/examples/JaccardIndex.java | 2 +- .../flink/graph/examples/TriangleListing.java | 2 +- .../annotate/directed/VertexDegrees.java | 2 +- .../directed/LocalClusteringCoefficient.java | 32 +- .../LocalClusteringCoefficient.java | 31 +- .../library/metric/directed/EdgeMetrics.java | 507 ++++ .../metric/directed/VertexMetrics.java | 102 +- .../metric/undirected/EdgeMetrics.java | 445 ++++ .../metric/undirected/VertexMetrics.java | 63 +- .../metric/directed/EdgeMetricsTest.java | 90 + .../metric/directed/VertexMetricsTest.java | 19 +- .../metric/undirected/EdgeMetricsTest.java | 89 + .../metric/undirected/VertexMetricsTest.java | 14 +- .../flink/api/scala/table/expressionDsl.scala | 20 +- .../flink/api/table/TableEnvironment.scala | 10 + .../api/table/codegen/CodeGenUtils.scala | 13 +- .../codegen/calls/FloorCeilCallGen.scala | 53 +- .../table/codegen/calls/ScalarFunctions.scala | 44 +- .../table/expressions/ExpressionParser.scala | 27 +- .../table/expressions/fieldExpression.scala | 11 +- .../flink/api/table/expressions/time.scala | 70 +- .../api/table/plan/RexNodeTranslator.scala | 17 + .../api/table/plan/logical/operators.scala | 6 +- .../flink/api/table/sinks/CsvTableSink.scala | 4 +- .../flink/api/table/sinks/TableSink.scala | 40 +- .../flink/api/table/sinks/TableSinkBase.scala | 67 + .../org/apache/flink/api/table/table.scala | 8 +- .../api/table/validate/FunctionCatalog.scala | 3 + .../api/java/batch/table/SelectITCase.java | 23 + .../api/scala/batch/table/SelectITCase.scala | 53 + .../expressions/ScalarFunctionsTest.scala | 123 + flink-mesos/pom.xml | 273 ++ .../java/org/apache/flink/mesos/Utils.java | 76 + .../flink/mesos/cli/FlinkMesosSessionCli.java | 72 + .../LaunchableMesosWorker.java | 204 ++ .../MesosApplicationMasterRunner.java | 603 +++++ .../clusterframework/MesosConfigKeys.java | 44 + .../MesosFlinkResourceManager.java | 762 ++++++ .../MesosTaskManagerParameters.java | 81 + .../MesosTaskManagerRunner.java | 117 + .../RegisteredMesosWorkerNode.scala | 33 + .../store/MesosWorkerStore.java | 221 ++ .../store/StandaloneMesosWorkerStore.java | 86 + .../store/ZooKeeperMesosWorkerStore.java | 302 +++ .../flink/mesos/scheduler/LaunchableTask.java | 42 + .../flink/mesos/scheduler/SchedulerProxy.java | 105 + .../mesos/scheduler/TaskSchedulerBuilder.java | 41 + .../scheduler/messages/AcceptOffers.java | 77 + .../mesos/scheduler/messages/Connected.java | 29 + .../scheduler/messages/Disconnected.java | 23 +- .../flink/mesos/scheduler/messages/Error.java | 47 + .../scheduler/messages/ExecutorLost.java | 66 + .../scheduler/messages/FrameworkMessage.java | 67 + .../scheduler/messages/OfferRescinded.java | 36 +- .../scheduler/messages/ReRegistered.java | 51 + .../mesos/scheduler/messages/Registered.java | 44 +- .../scheduler/messages/ResourceOffers.java | 52 + .../mesos/scheduler/messages/SlaveLost.java | 49 + .../scheduler/messages/StatusUpdate.java | 50 + .../flink/mesos/util/MesosArtifactServer.java | 304 +++ .../flink/mesos/util/MesosConfiguration.java | 151 ++ .../flink/mesos/util/ZooKeeperUtils.java | 40 + .../src/main/resources/log4j.properties | 27 + .../clusterframework/MesosJobManager.scala | 84 + .../clusterframework/MesosTaskManager.scala | 65 + .../mesos/scheduler/ConnectionMonitor.scala | 126 + .../mesos/scheduler/LaunchCoordinator.scala | 362 +++ .../scheduler/ReconciliationCoordinator.scala | 186 ++ .../flink/mesos/scheduler/TaskMonitor.scala | 260 ++ .../apache/flink/mesos/scheduler/Tasks.scala | 127 + .../MesosFlinkResourceManagerTest.java | 718 ++++++ .../src/test/resources/log4j-test.properties | 32 + .../src/test/resources/logback-test.xml | 37 + .../scala/org/apache/flink/mesos/Utils.scala | 54 + .../scheduler/LaunchCoordinatorTest.scala | 466 ++++ .../ReconciliationCoordinatorTest.scala | 233 ++ .../mesos/scheduler/TaskMonitorTest.scala | 257 ++ .../flink/mesos/scheduler/TasksTest.scala | 210 ++ .../apache/flink/runtime/akka/FSMSpec.scala | 59 + .../java/org/apache/flink/metrics/Meter.java | 51 + .../org/apache/flink/metrics/MetricGroup.java | 20 + .../groups/UnregisteredMetricsGroup.java | 11 + .../metrics/reporter/AbstractReporter.java | 6 + .../apache/flink/metrics/util/TestMeter.java | 42 + .../ScheduledDropwizardReporter.java | 19 +- .../metrics/DropwizardHistogramWrapper.java | 14 +- .../metrics/DropwizardMeterWrapper.java | 57 + .../dropwizard/metrics/FlinkMeterWrapper.java | 77 + .../ScheduledDropwizardReporterTest.java | 9 +- .../metrics/DropwizardMeterWrapperTest.java | 59 + .../metrics/FlinkMeterWrapperTest.java | 64 + .../apache/flink/metrics/jmx/JMXReporter.java | 28 + .../flink/metrics/jmx/JMXReporterTest.java | 42 + .../flink/metrics/statsd/StatsDReporter.java | 12 + .../metrics/statsd/StatsDReporterTest.java | 63 + .../optimizer/DistinctCompilationTest.java | 66 +- .../resources/archetype-resources/pom.xml | 1 - .../resources/archetype-resources/pom.xml | 1 - .../webmonitor/WebRuntimeMonitorITCase.java | 6 +- .../apache/flink/runtime/blob/BlobServer.java | 19 +- .../apache/flink/runtime/blob/BlobUtils.java | 3 +- .../runtime/blob/FileSystemBlobStore.java | 17 +- .../checkpoint/CheckpointCoordinator.java | 386 +-- .../CheckpointCoordinatorDeActivator.java | 45 +- .../checkpoint/CompletedCheckpoint.java | 56 +- .../runtime/checkpoint/KeyGroupState.java | 87 - .../runtime/checkpoint/PendingCheckpoint.java | 67 +- .../runtime/checkpoint/PendingSavepoint.java | 7 +- .../StandaloneCheckpointIDCounter.java | 7 +- .../StandaloneCheckpointRecoveryFactory.java | 4 +- .../StandaloneCompletedCheckpointStore.java | 20 +- .../runtime/checkpoint/SubtaskState.java | 56 +- .../flink/runtime/checkpoint/TaskState.java | 151 +- .../ZooKeeperCheckpointIDCounter.java | 4 +- .../ZooKeeperCheckpointRecoveryFactory.java | 4 +- .../ZooKeeperCompletedCheckpointStore.java | 50 +- .../savepoint/FsSavepointStore.java | 6 +- .../savepoint/HeapSavepointStore.java | 7 +- .../checkpoint/savepoint/Savepoint.java | 8 +- .../checkpoint/savepoint/SavepointLoader.java | 4 +- .../savepoint/SavepointSerializers.java | 7 +- .../checkpoint/savepoint/SavepointStore.java | 7 +- .../savepoint/SavepointV0Serializer.java | 186 -- .../{SavepointV0.java => SavepointV1.java} | 18 +- .../savepoint/SavepointV1Serializer.java | 232 ++ .../stats/SimpleCheckpointStatsTracker.java | 15 +- .../client/JobAttachmentClientActor.java | 172 ++ .../flink/runtime/client/JobClient.java | 292 ++- .../flink/runtime/client/JobClientActor.java | 281 +-- ...ientActorRegistrationTimeoutException.java | 35 + .../runtime/client/JobListeningContext.java | 145 ++ .../runtime/client/JobRetrievalException.java | 42 + .../client/JobSubmissionClientActor.java | 192 ++ .../InputChannelDeploymentDescriptor.java | 1 + .../InputGateDeploymentDescriptor.java | 1 + .../deployment/ResultPartitionLocation.java | 1 + .../deployment/TaskDeploymentDescriptor.java | 38 +- .../flink/runtime/execution/Environment.java | 17 +- .../runtime/executiongraph/Execution.java | 41 +- .../executiongraph/ExecutionGraph.java | 84 +- .../executiongraph/ExecutionJobVertex.java | 137 +- .../ExecutionStatusListener.java | 54 + .../executiongraph/ExecutionVertex.java | 31 +- .../executiongraph/JobStatusListener.java | 39 + .../StatusListenerMessenger.java | 70 + .../fs/hdfs/HadoopDataOutputStream.java | 5 + .../runtime/instance/HardwareDescription.java | 2 + .../io/async/AbstractAsyncIOCallable.java | 157 ++ .../runtime/io/async/AsyncDoneCallback.java | 17 +- .../runtime/io/async/AsyncStoppable.java | 47 + .../async/AsyncStoppableTaskWithCallback.java | 55 + .../io/async/StoppableCallbackCallable.java | 30 + .../flink/runtime/jobgraph/JobVertex.java | 31 +- .../runtime/jobgraph/tasks/StatefulTask.java | 14 +- ...eryMode.java => HighAvailabilityMode.java} | 46 +- .../StandaloneSubmittedJobGraphStore.java | 2 +- .../ZooKeeperSubmittedJobGraphStore.java | 22 +- .../checkpoint/AcknowledgeCheckpoint.java | 53 +- .../metrics/groups/AbstractMetricGroup.java | 12 + .../metrics/groups/ProxyMetricGroup.java | 11 + .../flink/runtime/net/ConnectionUtils.java | 46 +- .../flink/runtime/query/KvStateRegistry.java | 6 +- .../runtime/query/TaskKvStateRegistry.java | 2 +- .../query/netty/KvStateServerHandler.java | 8 +- .../state/AbstractCloseableHandle.java | 23 +- .../runtime/state/AbstractHeapState.java | 220 -- .../runtime/state/AbstractStateBackend.java | 520 +--- .../state/AsynchronousKvStateSnapshot.java | 68 - .../runtime/state/ChainedStateHandle.java | 131 + .../state/CheckpointStreamFactory.java | 67 + .../flink/runtime/state/DoneFuture.java | 70 + .../runtime/state/GenericFoldingState.java | 74 +- .../flink/runtime/state/GenericListState.java | 69 +- .../runtime/state/GenericReducingState.java | 74 +- .../flink/runtime/state/KeyGroupRange.java | 178 ++ .../state/KeyGroupRangeAssignment.java | 97 + .../runtime/state/KeyGroupRangeOffsets.java | 203 ++ .../runtime/state/KeyGroupsStateHandle.java | 163 ++ .../runtime/state/KeyedStateBackend.java | 335 +++ .../apache/flink/runtime/state/KvState.java | 39 +- .../flink/runtime/state/KvStateSnapshot.java | 61 - .../runtime/state/RetrievableStateHandle.java | 32 + .../state/RetrievableStreamStateHandle.java | 82 + .../runtime/state/StateBackendFactory.java | 4 +- .../flink/runtime/state/StateObject.java | 5 +- .../apache/flink/runtime/state/StateUtil.java | 101 + .../flink/runtime/state/StateUtils.java | 59 - .../runtime/state/StreamStateHandle.java | 20 +- .../state/filesystem/AbstractFsState.java | 95 - .../filesystem/AbstractFsStateSnapshot.java | 138 -- .../FileSerializableStateHandle.java | 72 - ...eStateHandle.java => FileStateHandle.java} | 78 +- .../filesystem/FileStreamStateHandle.java | 83 - .../filesystem/FsCheckpointStreamFactory.java | 315 +++ .../state/filesystem/FsFoldingState.java | 161 -- .../runtime/state/filesystem/FsListState.java | 149 -- .../state/filesystem/FsReducingState.java | 165 -- .../state/filesystem/FsStateBackend.java | 386 +-- .../state/filesystem/FsValueState.java | 148 -- .../runtime/state/heap/AbstractHeapState.java | 187 ++ .../runtime/state/heap/HeapFoldingState.java | 124 + .../state/heap/HeapKeyedStateBackend.java | 327 +++ .../runtime/state/heap/HeapListState.java | 157 ++ .../runtime/state/heap/HeapReducingState.java | 123 + .../runtime/state/heap/HeapValueState.java | 112 + .../flink/runtime/state/heap/StateTable.java | 77 + .../state/memory/AbstractMemState.java | 82 - .../memory/AbstractMemStateSnapshot.java | 144 -- .../state/memory/ByteStreamStateHandle.java | 84 +- .../memory/MemCheckpointStreamFactory.java | 146 ++ .../runtime/state/memory/MemFoldingState.java | 135 - .../runtime/state/memory/MemListState.java | 120 - .../state/memory/MemReducingState.java | 139 -- .../runtime/state/memory/MemValueState.java | 122 - .../state/memory/MemoryStateBackend.java | 191 +- .../state/memory/SerializedStateHandle.java | 87 - .../taskmanager/RuntimeEnvironment.java | 42 +- .../flink/runtime/taskmanager/Task.java | 51 +- .../runtime/util/EnvironmentInformation.java | 35 +- .../runtime/{instance => util}/Hardware.java | 89 +- .../runtime/util/LeaderRetrievalUtils.java | 36 +- .../runtime/util/SerializableArrayList.java | 100 - .../runtime/util/SerializableHashMap.java | 111 - .../runtime/util/SerializableHashSet.java | 78 - .../flink/runtime/util/ZooKeeperUtils.java | 143 +- ...ava => RetrievableStateStorageHelper.java} | 8 +- .../zookeeper/ZooKeeperStateHandleStore.java | 78 +- .../FileSystemStateStorageHelper.java | 17 +- .../ContaineredJobManager.scala | 173 ++ .../flink/runtime/jobmanager/JobInfo.scala | 62 +- .../flink/runtime/jobmanager/JobManager.scala | 218 +- .../runtime/messages/JobClientMessages.scala | 23 +- .../runtime/messages/JobManagerMessages.scala | 54 +- .../minicluster/FlinkMiniCluster.scala | 30 +- .../runtime/taskmanager/TaskManager.scala | 25 +- .../testingUtils/TestingJobManagerLike.scala | 15 +- .../TestingJobManagerMessages.scala | 17 + .../runtime/blob/BlobRecoveryITCase.java | 8 +- .../checkpoint/CheckpointCoordinatorTest.java | 749 +++++- .../CheckpointStateRestoreTest.java | 49 +- .../CompletedCheckpointStoreTest.java | 25 +- .../checkpoint/CompletedCheckpointTest.java | 8 +- .../checkpoint/CoordinatorShutdownTest.java | 3 +- ...ecutionGraphCheckpointCoordinatorTest.java | 4 - .../checkpoint/PendingCheckpointTest.java | 17 +- .../checkpoint/PendingSavepointTest.java | 15 +- ...oKeeperCompletedCheckpointStoreITCase.java | 44 +- .../savepoint/FsSavepointStoreTest.java | 8 +- .../savepoint/SavepointLoaderTest.java | 12 +- ...st.java => SavepointV1SerializerTest.java} | 8 +- ...epointV0Test.java => SavepointV1Test.java} | 33 +- .../SimpleCheckpointStatsTrackerTest.java | 52 +- .../client/JobClientActorRecoveryITCase.java | 3 +- .../runtime/client/JobClientActorTest.java | 190 +- .../TaskDeploymentDescriptorTest.java | 4 +- .../runtime/event/task/TaskEventTest.java | 27 - .../BlobLibraryCacheRecoveryITCase.java | 8 +- .../executiongraph/LocalInputSplitsTest.java | 436 ---- .../jobmanager/HighAvailabilityModeTest.java | 71 + .../jobmanager/JobManagerHARecoveryTest.java | 106 +- ...ooKeeperSubmittedJobGraphsStoreITCase.java | 21 +- .../scheduler/SchedulerTestUtils.java | 2 + .../JobManagerLeaderElectionTest.java | 8 +- .../LeaderChangeJobRecoveryTest.java | 73 +- .../ZooKeeperLeaderElectionTest.java | 28 +- .../ZooKeeperLeaderRetrievalTest.java | 8 +- .../messages/CheckpointMessagesTest.java | 56 +- .../metrics/groups/TaskManagerGroupTest.java | 14 +- .../runtime/net/ConnectionUtilsTest.java | 53 +- .../operators/testutils/DummyEnvironment.java | 25 +- .../testutils/DummyEnvironment.java.orig | 185 -- .../operators/testutils/MockEnvironment.java | 33 +- .../query/QueryableStateClientTest.java | 35 +- .../query/netty/KvStateClientTest.java | 42 +- .../query/netty/KvStateServerHandlerTest.java | 228 +- .../query/netty/KvStateServerTest.java | 53 +- .../state/AbstractCloseableHandleTest.java | 10 + .../runtime/state/FileStateBackendTest.java | 159 +- .../state/KeyGroupRangeOffsetTest.java | 136 + .../runtime/state/KeyGroupRangeTest.java | 101 + .../runtime/state/MemoryStateBackendTest.java | 46 +- .../runtime/state/StateBackendTestBase.java | 668 +++-- .../FsCheckpointStateOutputStreamTest.java | 51 +- .../taskmanager/TaskAsyncCallTest.java | 18 +- .../runtime/taskmanager/TaskManagerTest.java | 27 +- .../flink/runtime/taskmanager/TaskTest.java | 2 +- .../runtime/testutils/JobManagerProcess.java | 2 +- .../runtime/testutils/TaskManagerProcess.java | 2 +- .../runtime/testutils/ZooKeeperTestUtils.java | 28 +- .../{instance => util}/HardwareTest.java | 4 +- .../flink/runtime/util/ZooKeeperUtilTest.java | 2 +- .../ZooKeeperStateHandleStoreITCase.java | 62 +- .../zookeeper/ZooKeeperTestEnvironment.java | 6 +- .../runtime/testingUtils/TestingUtils.scala | 6 +- .../cassandra/CassandraCommitter.java | 13 +- .../cassandra/CassandraPojoSink.java | 12 +- ...est.java => CassandraConnectorITCase.java} | 76 +- .../examples/ElasticsearchExample.java | 3 +- .../flink-connector-elasticsearch2/pom.xml | 2 +- .../examples/ElasticsearchExample.java | 11 +- .../flink-connector-filesystem/pom.xml | 9 + .../connectors/fs/AvroKeyValueSinkWriter.java | 6 +- .../streaming/connectors/fs/Bucketer.java | 3 + .../connectors/fs/DateTimeBucketer.java | 2 + .../connectors/fs/NonRollingBucketer.java | 6 +- .../streaming/connectors/fs/RollingSink.java | 8 +- .../connectors/fs/SequenceFileWriter.java | 3 +- .../connectors/fs/StreamWriterBase.java | 3 +- .../flink/streaming/connectors/fs/Writer.java | 3 +- .../fs/bucketing/BasePathBucketer.java | 39 + .../connectors/fs/bucketing/Bucketer.java | 47 + .../fs/bucketing/BucketingSink.java | 1034 ++++++++ .../fs/bucketing/DateTimeBucketer.java | 102 + .../fs/RollingSinkFaultTolerance2ITCase.java | 3 + .../fs/RollingSinkFaultToleranceITCase.java | 3 + .../connectors/fs/RollingSinkITCase.java | 3 + .../BucketingSinkFaultTolerance2ITCase.java | 289 +++ .../BucketingSinkFaultToleranceITCase.java | 282 +++ .../fs/bucketing/BucketingSinkTest.java | 511 ++++ .../kafka/Kafka08JsonTableSink.java | 51 + .../kafka/Kafka08JsonTableSinkITCase.java | 40 + .../kafka/Kafka09JsonTableSink.java | 50 + .../kafka/Kafka09JsonTableSinkITCase.java | 39 + .../connectors/kafka/KafkaJsonTableSink.java | 47 + .../connectors/kafka/KafkaTableSink.java | 126 + .../connectors/kafka/KafkaTableSource.java | 15 +- .../connectors/kafka/internals/TypeUtil.java | 38 + .../JsonRowDeserializationSchema.java | 4 +- .../JsonRowSerializationSchema.java | 70 + .../kafka/JsonRowSerializationSchemaTest.java | 98 + .../kafka/KafkaTableSinkTestBase.java | 142 ++ .../kinesis/FlinkKinesisConsumer.java | 9 +- .../config/ConsumerConfigConstants.java | 7 + .../kinesis/internals/KinesisDataFetcher.java | 129 +- .../kinesis/internals/ShardConsumer.java | 78 +- .../kinesis/proxy/KinesisProxy.java | 8 +- .../kinesis/util/KinesisConfigUtil.java | 30 +- .../kinesis/internals/ShardConsumerTest.java | 40 + .../FakeKinesisBehavioursFactory.java | 66 +- .../api/datastream/CoGroupedStreams.java | 44 + .../api/datastream/JoinedStreams.java | 85 + .../streaming/api/datastream/KeyedStream.java | 11 +- .../SingleOutputStreamOperator.java | 19 + .../StreamExecutionEnvironment.java | 112 +- .../ContinuousFileMonitoringFunction.java | 8 +- .../source/ContinuousFileReaderOperator.java | 35 +- .../streaming/api/graph/StreamConfig.java | 45 +- .../streaming/api/graph/StreamGraph.java | 33 +- .../api/graph/StreamGraphGenerator.java | 44 +- .../flink/streaming/api/graph/StreamNode.java | 25 + .../api/graph/StreamingJobGraphGenerator.java | 26 +- .../api/operators/AbstractStreamOperator.java | 135 +- .../operators/AbstractUdfStreamOperator.java | 76 +- .../api/operators/StreamOperator.java | 21 +- .../transformations/StreamTransformation.java | 24 + .../api/windowing/evictors/TimeEvictor.java | 2 +- .../triggers/ContinuousEventTimeTrigger.java | 3 +- .../ContinuousProcessingTimeTrigger.java | 3 +- .../windowing/triggers/PurgingTrigger.java | 2 +- .../operators/GenericWriteAheadSink.java | 96 +- ...ctAlignedProcessingTimeWindowOperator.java | 53 +- .../windowing/EvictingWindowOperator.java | 2 +- .../operators/windowing/WindowOperator.java | 36 +- .../ConfigurableStreamPartitioner.java | 39 + ...er.java => KeyGroupStreamPartitioner.java} | 41 +- .../streaming/runtime/tasks/StreamTask.java | 386 ++- .../runtime/tasks/StreamTaskState.java | 185 -- .../runtime/tasks/StreamTaskStateList.java | 123 - .../api/AggregationFunctionTest.java | 8 +- .../flink/streaming/api/DataStreamTest.java | 8 +- .../streaming/api/RestartStrategyTest.java | 4 +- .../api/graph/SlotAllocationTest.java | 3 +- .../api/graph/StreamGraphGeneratorTest.java | 197 ++ .../graph/StreamingJobGraphGeneratorTest.java | 16 +- .../api/operators/StreamGroupedFoldTest.java | 12 +- .../operators/StreamGroupedReduceTest.java | 9 +- .../StreamingRuntimeContextTest.java | 29 +- ...treamingJobGraphGeneratorNodeHashTest.java | 64 +- .../operators/GenericWriteAheadSinkTest.java | 68 +- .../operators/StreamOperatorChainingTest.java | 15 - .../operators/StreamSourceOperatorTest.java | 2 + .../operators/WriteAheadSinkTestBase.java | 163 +- ...ignedProcessingTimeWindowOperatorTest.java | 296 +-- ...ignedProcessingTimeWindowOperatorTest.java | 329 ++- .../windowing/AllWindowTranslationTest.java | 4 + .../windowing/EvictingWindowOperatorTest.java | 13 +- .../windowing/WindowOperatorTest.java | 198 +- .../windowing/WindowTranslationTest.java | 10 + .../windowing/WindowingTestHarnessTest.java | 6 +- ...ava => KeyGroupStreamPartitionerTest.java} | 28 +- .../partitioner/RescalePartitionerTest.java | 4 + .../tasks/InterruptSensitiveRestoreTest.java | 65 +- .../runtime/tasks/OneInputStreamTaskTest.java | 290 ++- .../tasks/OneInputStreamTaskTestHarness.java | 1 + .../runtime/tasks/StreamMockEnvironment.java | 11 +- .../tasks/StreamTaskAsyncCheckpointTest.java | 470 ++-- .../runtime/tasks/StreamTaskTest.java | 178 +- .../runtime/tasks/StreamTaskTestHarness.java | 37 +- ...eyedOneInputStreamOperatorTestHarness.java | 206 ++ .../flink/streaming/util/MockContext.java | 76 +- .../OneInputStreamOperatorTestHarness.java | 119 +- .../flink/streaming/util/TestHarnessUtil.java | 5 +- .../streaming/util/WindowingTestHarness.java | 15 +- .../src/test/resources/log4j-test.properties | 2 +- .../streaming/api/scala/DataStream.scala | 11 + .../scala/StreamExecutionEnvironment.scala | 61 +- .../streaming/api/scala/DataStreamTest.scala | 2 +- .../api/scala/StreamingOperatorsITCase.scala | 1 + .../flink/core/testutils/OneShotLatch.java | 55 + .../core/testutils/OneShotLatchTest.java | 55 + .../apache/flink/test/util/TestBaseUtils.java | 2 +- .../test/util/ForkableFlinkMiniCluster.scala | 10 +- ...tinuousFileProcessingCheckpointITCase.java | 5 +- ...EventTimeAllWindowCheckpointingITCase.java | 2 +- .../EventTimeWindowCheckpointingITCase.java | 29 +- .../test/checkpointing/RescalingITCase.java | 675 +++++ .../test/checkpointing/SavepointITCase.java | 44 +- .../WindowCheckpointingITCase.java | 14 +- .../test/classloading/ClassLoaderITCase.java | 4 +- .../jar/CustomKvStateProgram.java | 37 +- .../clients/examples/JobRetrievalITCase.java | 140 ++ .../CustomDistributionITCase.java | 110 +- .../test/query/QueryableStateITCase.java | 2 + .../test/recovery/ChaosMonkeyITCase.java | 4 +- .../JobManagerHACheckpointRecoveryITCase.java | 4 +- .../JobManagerHAJobGraphRecoveryITCase.java | 10 +- ...erHAProcessFailureBatchRecoveryITCase.java | 6 +- .../ZooKeeperLeaderElectionITCase.java | 8 +- .../state/StateHandleSerializationTest.java | 22 +- .../api/StreamingOperatorsITCase.java | 17 + .../runtime/DataStreamPojoITCase.java | 2 + .../test/streaming/runtime/IterateITCase.java | 1 + .../streaming/runtime/StateBackendITCase.java | 77 +- .../streaming/runtime/TimestampITCase.java | 1 + .../flink/test/web/WebFrontendITCase.java | 3 +- .../translation/CustomPartitioningTest.scala | 5 +- .../DeltaIterationTranslationTest.scala | 1 + .../translation/DistinctTranslationTest.scala | 52 - ...iFrontendYarnAddressConfigurationTest.java | 4 +- .../flink/yarn/FlinkYarnSessionCliTest.java | 2 +- .../yarn/YARNHighAvailabilityITCase.java | 2 +- .../org/apache/flink/yarn/YarnTestBase.java | 92 +- .../yarn/AbstractYarnClusterDescriptor.java | 8 +- .../yarn/YarnApplicationMasterRunner.java | 18 +- .../apache/flink/yarn/YarnClusterClient.java | 159 +- .../flink/yarn/cli/FlinkYarnSessionCli.java | 7 +- .../apache/flink/yarn/ApplicationClient.scala | 27 - .../apache/flink/yarn/YarnJobManager.scala | 104 +- .../org/apache/flink/yarn/YarnMessages.scala | 7 - pom.xml | 6 +- tools/maven/checkstyle.xml | 6 + tools/travis_mvn_watchdog.sh | 14 +- 758 files changed, 35128 insertions(+), 13387 deletions(-) delete mode 100644 docs/_includes/navbar.html create mode 100644 docs/_includes/sidenav.html rename docs/{apis/programming_guide.md => _layouts/redirect.html} (71%) delete mode 100644 docs/apis/batch/fig/LICENSE.txt delete mode 100644 docs/apis/batch/libs/gelly.md delete mode 100644 docs/apis/batch/libs/table.md delete mode 100644 docs/apis/streaming/fig/LICENSE.txt delete mode 100644 docs/apis/streaming_guide.md rename docs/concepts/{concepts.md => index.md} (88%) rename docs/{apis/common/index.md => dev/api_concepts.md} (97%) rename docs/{apis/index.md => dev/apis.md} (93%) rename docs/{apis => dev}/batch/connectors.md (98%) rename docs/{apis => dev}/batch/dataset_transformations.md (99%) rename docs/{apis => dev}/batch/examples.md (99%) rename docs/{apis => dev}/batch/fault_tolerance.md (98%) rename docs/{apis => dev}/batch/hadoop_compatibility.md (98%) rename docs/{apis => dev}/batch/index.md (97%) rename docs/{apis => dev}/batch/iterations.md (98%) rename docs/{apis => dev}/batch/python.md (98%) rename docs/{apis => dev}/batch/zip_elements_guide.md (97%) rename docs/{apis => dev}/cluster_execution.md (97%) rename docs/{apis/streaming => dev}/connectors/cassandra.md (95%) rename docs/{apis/streaming => dev}/connectors/elasticsearch.md (96%) rename docs/{apis/streaming => dev}/connectors/elasticsearch2.md (95%) rename docs/{apis/streaming => dev}/connectors/filesystem_sink.md (54%) rename docs/{apis/streaming => dev}/connectors/index.md (91%) rename docs/{apis/streaming => dev}/connectors/kafka.md (93%) rename docs/{apis/streaming => dev}/connectors/kinesis.md (94%) rename docs/{apis/streaming => dev}/connectors/nifi.md (93%) rename docs/{apis/streaming => dev}/connectors/rabbitmq.md (95%) rename docs/{apis/streaming => dev}/connectors/redis.md (95%) rename docs/{apis/streaming => dev}/connectors/twitter.md (89%) rename docs/{apis/streaming/index.md => dev/datastream_api.md} (97%) rename docs/{apis/streaming => dev}/event_time.md (93%) rename docs/{apis/streaming => dev}/event_timestamp_extractors.md (88%) rename docs/{apis/streaming => dev}/event_timestamps_watermarks.md (97%) rename docs/{apis/batch/libs => dev}/index.md (76%) rename docs/{apis => dev}/java8.md (97%) rename docs/{apis/streaming/libs/index.md => dev/libraries.md} (83%) rename docs/{apis/streaming => dev}/libs/cep.md (98%) rename docs/{apis/batch => dev}/libs/gelly/graph_algorithms.md (99%) rename docs/{apis/batch => dev}/libs/gelly/graph_api.md (98%) rename docs/{apis/batch => dev}/libs/gelly/graph_generators.md (99%) rename docs/{apis/batch => dev}/libs/gelly/index.md (90%) rename docs/{apis/batch => dev}/libs/gelly/iterative_graph_processing.md (97%) rename docs/{apis/batch => dev}/libs/gelly/library_methods.md (99%) rename docs/{apis/batch => dev}/libs/ml/als.md (94%) rename docs/{apis/batch => dev}/libs/ml/contribution_guide.md (98%) rename docs/{apis/batch => dev}/libs/ml/cross_validation.md (98%) rename docs/{apis/batch => dev}/libs/ml/distance_metrics.md (96%) rename docs/{apis/batch => dev}/libs/ml/index.md (92%) rename docs/{apis/batch => dev}/libs/ml/knn.md (95%) rename docs/{apis/batch => dev}/libs/ml/min_max_scaler.md (97%) rename docs/{apis/batch => dev}/libs/ml/multiple_linear_regression.md (96%) rename docs/{apis/batch => dev}/libs/ml/optimization.md (99%) rename docs/{apis/batch => dev}/libs/ml/pipelines.md (96%) rename docs/{apis/batch => dev}/libs/ml/polynomial_features.md (94%) rename docs/{apis/batch => dev}/libs/ml/quickstart.md (95%) rename docs/{apis/batch => dev}/libs/ml/standard_scaler.md (87%) rename docs/{apis/batch => dev}/libs/ml/svm.md (98%) rename docs/{apis/streaming => dev/libs}/storm_compatibility.md (97%) rename docs/{apis => dev}/local_execution.md (95%) rename docs/{apis/batch/libs/fig/LICENSE.txt => dev/quickstarts.md} (77%) rename docs/{apis => dev}/scala_api_extensions.md (97%) rename docs/{apis => dev}/scala_shell.md (98%) rename docs/{apis/streaming => dev}/state.md (98%) rename docs/{apis/streaming => dev}/state_backends.md (94%) rename docs/{apis/table.md => dev/table_api.md} (96%) rename docs/{internals => dev}/types_serialization.md (97%) rename docs/{apis/streaming => dev}/windows.md (80%) rename docs/{internals => }/fig/ClientJmTm.svg (100%) rename docs/{setup => }/fig/FlinkOnYarn.svg (100%) rename docs/{internals => }/fig/back_pressure_sampling.png (100%) rename docs/{internals => }/fig/back_pressure_sampling_high.png (100%) rename docs/{internals => }/fig/back_pressure_sampling_in_progress.png (100%) rename docs/{internals => }/fig/back_pressure_sampling_ok.png (100%) rename docs/{internals => }/fig/checkpointing.svg (100%) rename docs/{concepts => }/fig/checkpoints.svg (100%) rename docs/{concepts => }/fig/event_ingestion_processing_time.svg (100%) rename docs/{setup => }/fig/flink-on-emr.png (100%) rename docs/{apis/batch/libs => }/fig/gelly-example-graph.png (100%) rename docs/{apis/batch/libs => }/fig/gelly-filter.png (100%) rename docs/{apis/batch/libs => }/fig/gelly-gsa-sssp1.png (100%) rename docs/{apis/batch/libs => }/fig/gelly-reduceOnEdges.png (100%) rename docs/{apis/batch/libs => }/fig/gelly-reduceOnNeighbors.png (100%) rename docs/{apis/batch/libs => }/fig/gelly-union.png (100%) rename docs/{apis/batch/libs => }/fig/gelly-vc-sssp1.png (100%) rename docs/{apis/batch => }/fig/iterations_delta_iterate_operator.png (100%) rename docs/{apis/batch => }/fig/iterations_delta_iterate_operator_example.png (100%) rename docs/{apis/batch => }/fig/iterations_iterate_operator.png (100%) rename docs/{apis/batch => }/fig/iterations_iterate_operator_example.png (100%) rename docs/{apis/batch => }/fig/iterations_supersteps.png (100%) rename docs/{internals => }/fig/job_and_execution_graph.svg (100%) rename docs/{internals => }/fig/job_status.svg (100%) rename docs/{setup => }/fig/jobmanager_ha_overview.png (100%) rename docs/{apis/streaming => fig}/non-windowed.svg (100%) rename docs/{concepts => }/fig/parallel_dataflow.svg (100%) rename docs/{apis/streaming => }/fig/parallel_streams_watermarks.svg (100%) rename docs/{apis/common => }/fig/plan_visualizer.png (100%) rename docs/{concepts => }/fig/processes.svg (100%) rename docs/{concepts => }/fig/program_dataflow.svg (100%) rename docs/{internals => }/fig/projects_dependencies.svg (100%) rename docs/{apis/streaming => }/fig/rescale.svg (100%) rename docs/{apis/streaming => }/fig/savepoints-overview.png (100%) rename docs/{apis/streaming => }/fig/savepoints-program_ids.png (100%) rename docs/{apis/streaming => fig}/session-windows.svg (100%) rename docs/{apis/streaming => fig}/sliding-windows.svg (100%) rename docs/{concepts => }/fig/slot_sharing.svg (100%) rename docs/{internals => }/fig/slots.svg (100%) rename docs/{setup => }/fig/slots_parallelism.svg (100%) rename docs/{internals => }/fig/stack.svg (100%) rename docs/{internals => }/fig/state_machine.svg (100%) rename docs/{concepts => }/fig/state_partitioning.svg (100%) rename docs/{internals => }/fig/stream_aligning.svg (100%) rename docs/{internals => }/fig/stream_barriers.svg (100%) rename docs/{apis/streaming => }/fig/stream_watermark_in_order.svg (100%) rename docs/{apis/streaming => }/fig/stream_watermark_out_of_order.svg (100%) rename docs/{concepts => }/fig/tasks_chains.svg (100%) rename docs/{concepts => }/fig/tasks_slots.svg (100%) rename docs/{apis/streaming => }/fig/times_clocks.svg (100%) rename docs/{apis/streaming => fig}/tumbling-windows.svg (100%) rename docs/{apis/batch/libs => }/fig/vertex-centric supersteps.png (100%) rename docs/{concepts => }/fig/windows.svg (100%) delete mode 100644 docs/internals/_draft_distributed_akka.md delete mode 100644 docs/internals/coding_guidelines.md delete mode 100644 docs/internals/fig/LICENSE.txt delete mode 100644 docs/internals/how_to_contribute.md delete mode 100644 docs/libs/cep/index.md delete mode 100644 docs/libs/gelly_guide.md delete mode 100644 docs/libs/index.md delete mode 100644 docs/libs/ml/als.md delete mode 100644 docs/libs/ml/contribution_guide.md delete mode 100644 docs/libs/ml/distance_metrics.md delete mode 100644 docs/libs/ml/index.md delete mode 100644 docs/libs/ml/min_max_scaler.md delete mode 100644 docs/libs/ml/multiple_linear_regression.md delete mode 100644 docs/libs/ml/optimization.md delete mode 100644 docs/libs/ml/pipelines.md delete mode 100644 docs/libs/ml/polynomial_features.md delete mode 100644 docs/libs/ml/quickstart.md delete mode 100644 docs/libs/ml/standard_scaler.md delete mode 100644 docs/libs/ml/svm.md delete mode 100644 docs/libs/table.md rename docs/{internals/back_pressure_monitoring.md => monitoring/back_pressure.md} (89%) rename docs/{apis => monitoring}/best_practices.md (97%) create mode 100644 docs/monitoring/index.md rename docs/{internals => monitoring}/logging.md (92%) rename docs/{apis => monitoring}/metrics.md (88%) rename docs/{internals/monitoring_rest_api.md => monitoring/rest_api.md} (99%) create mode 100644 docs/page/font-awesome/css/font-awesome.css create mode 100644 docs/page/font-awesome/css/font-awesome.min.css create mode 100644 docs/page/font-awesome/fonts/FontAwesome.otf create mode 100644 docs/page/font-awesome/fonts/fontawesome-webfont.eot create mode 100644 docs/page/font-awesome/fonts/fontawesome-webfont.svg create mode 100644 docs/page/font-awesome/fonts/fontawesome-webfont.ttf create mode 100644 docs/page/font-awesome/fonts/fontawesome-webfont.woff create mode 100644 docs/page/font-awesome/fonts/fontawesome-webfont.woff2 create mode 100644 docs/redirects/back_pressure.md create mode 100644 docs/redirects/basic_api_concepts.md create mode 100644 docs/redirects/best_practices.md create mode 100644 docs/redirects/cassandra.md create mode 100644 docs/redirects/cep.md create mode 100644 docs/redirects/cli.md create mode 100644 docs/redirects/cluster_execution.md create mode 100644 docs/redirects/concepts.md rename docs/{apis => redirects}/connectors.md (81%) create mode 100644 docs/redirects/datastream_api.md create mode 100644 docs/redirects/elasticsearch.md create mode 100644 docs/redirects/elasticsearch2.md create mode 100644 docs/redirects/event_time.md create mode 100644 docs/redirects/event_timestamp_extractors.md create mode 100644 docs/redirects/event_timestamps_watermarks.md create mode 100644 docs/redirects/fault_tolerance.md create mode 100644 docs/redirects/filesystem_sink.md create mode 100644 docs/redirects/gelly.md create mode 100644 docs/redirects/java8.md create mode 100644 docs/redirects/kafka.md create mode 100644 docs/redirects/kinesis.md create mode 100644 docs/redirects/local_execution.md create mode 100644 docs/redirects/metrics.md create mode 100644 docs/redirects/ml.md create mode 100644 docs/redirects/programming_guide.md create mode 100644 docs/redirects/rest_api.md create mode 100644 docs/redirects/savepoints.md create mode 100644 docs/redirects/scala_api_extensions.md create mode 100644 docs/redirects/scala_shell.md create mode 100644 docs/redirects/state.md create mode 100644 docs/redirects/state_backends.md create mode 100644 docs/redirects/storm_compat.md create mode 100644 docs/redirects/streaming_guide.md create mode 100644 docs/redirects/table.md create mode 100644 docs/redirects/types_serialization.md create mode 100644 docs/redirects/windows.md rename docs/{apis => setup}/cli.md (98%) create mode 100644 docs/setup/deployment.md rename docs/{apis/streaming => setup}/fault_tolerance.md (96%) delete mode 100644 docs/setup/fig/LICENSE.txt rename docs/{apis/streaming => setup}/savepoints.md (89%) create mode 100644 flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java delete mode 100644 flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/FullyAsyncRocksDBStateBackendTest.java rename flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/{RocksDBAsyncKVSnapshotTest.java => RocksDBAsyncSnapshotTest.java} (62%) create mode 100644 flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBMergeIteratorTest.java rename {flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source => flink-core/src/main/java/org/apache/flink/api/common/io}/FilePathFilter.java (91%) create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/io/GlobFilePathFilter.java create mode 100644 flink-core/src/main/java/org/apache/flink/core/memory/ByteArrayOutputStreamWithPos.java create mode 100644 flink-core/src/main/java/org/apache/flink/util/ConfigurationUtil.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/common/io/DefaultFilterTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/common/io/GlobFilePathFilterTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/util/ConfigurationUtilTest.java create mode 100644 flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/driver/GraphMetrics.java create mode 100644 flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/metric/directed/EdgeMetrics.java create mode 100644 flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/library/metric/undirected/EdgeMetrics.java create mode 100644 flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/metric/directed/EdgeMetricsTest.java create mode 100644 flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/library/metric/undirected/EdgeMetricsTest.java create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sinks/TableSinkBase.scala create mode 100644 flink-mesos/pom.xml create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/Utils.java create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/cli/FlinkMesosSessionCli.java create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosConfigKeys.java create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerRunner.java create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/RegisteredMesosWorkerNode.scala create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/MesosWorkerStore.java create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/StandaloneMesosWorkerStore.java create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/LaunchableTask.java create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/SchedulerProxy.java create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/TaskSchedulerBuilder.java create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/AcceptOffers.java create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Connected.java rename flink-runtime/src/test/java/org/apache/flink/runtime/event/task/EventList.java => flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Disconnected.java (63%) create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Error.java create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ExecutorLost.java create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/FrameworkMessage.java rename flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateHandle.java => flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/OfferRescinded.java (57%) create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ReRegistered.java rename flink-runtime/src/main/java/org/apache/flink/runtime/state/AsynchronousStateHandle.java => flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Registered.java (50%) create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ResourceOffers.java create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/SlaveLost.java create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/StatusUpdate.java create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosArtifactServer.java create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosConfiguration.java create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/util/ZooKeeperUtils.java create mode 100644 flink-mesos/src/main/resources/log4j.properties create mode 100644 flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosJobManager.scala create mode 100644 flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala create mode 100644 flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/ConnectionMonitor.scala create mode 100644 flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/LaunchCoordinator.scala create mode 100644 flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinator.scala create mode 100644 flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/TaskMonitor.scala create mode 100644 flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/Tasks.scala create mode 100644 flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java create mode 100644 flink-mesos/src/test/resources/log4j-test.properties create mode 100644 flink-mesos/src/test/resources/logback-test.xml create mode 100644 flink-mesos/src/test/scala/org/apache/flink/mesos/Utils.scala create mode 100644 flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/LaunchCoordinatorTest.scala create mode 100644 flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinatorTest.scala create mode 100644 flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/TaskMonitorTest.scala create mode 100644 flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/TasksTest.scala create mode 100644 flink-mesos/src/test/scala/org/apache/flink/runtime/akka/FSMSpec.scala create mode 100644 flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/Meter.java create mode 100644 flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/util/TestMeter.java create mode 100644 flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/DropwizardMeterWrapper.java create mode 100644 flink-metrics/flink-metrics-dropwizard/src/main/java/org/apache/flink/dropwizard/metrics/FlinkMeterWrapper.java create mode 100644 flink-metrics/flink-metrics-dropwizard/src/test/java/org/apache/flink/dropwizard/metrics/DropwizardMeterWrapperTest.java create mode 100644 flink-metrics/flink-metrics-dropwizard/src/test/java/org/apache/flink/dropwizard/metrics/FlinkMeterWrapperTest.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/KeyGroupState.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV0Serializer.java rename flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/{SavepointV0.java => SavepointV1.java} (81%) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV1Serializer.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/client/JobAttachmentClientActor.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorRegistrationTimeoutException.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/client/JobListeningContext.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/client/JobRetrievalException.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/client/JobSubmissionClientActor.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionStatusListener.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/JobStatusListener.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/StatusListenerMessenger.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/async/AbstractAsyncIOCallable.java rename flink-core/src/main/java/org/apache/flink/api/common/io/StrictlyLocalAssignment.java => flink-runtime/src/main/java/org/apache/flink/runtime/io/async/AsyncDoneCallback.java (75%) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/async/AsyncStoppable.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/async/AsyncStoppableTaskWithCallback.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/async/StoppableCallbackCallable.java rename flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/{RecoveryMode.java => HighAvailabilityMode.java} (67%) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractHeapState.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/AsynchronousKvStateSnapshot.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/ChainedStateHandle.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStreamFactory.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/DoneFuture.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupRange.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupRangeAssignment.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupRangeOffsets.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupsStateHandle.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateBackend.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/KvStateSnapshot.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/RetrievableStateHandle.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/RetrievableStreamStateHandle.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/StateUtil.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/StateUtils.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFsState.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/AbstractFsStateSnapshot.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FileSerializableStateHandle.java rename flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/{AbstractFileStateHandle.java => FileStateHandle.java} (65%) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FileStreamStateHandle.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStreamFactory.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsFoldingState.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsListState.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsReducingState.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsValueState.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/AbstractHeapState.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapFoldingState.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapListState.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapReducingState.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapValueState.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateTable.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/AbstractMemState.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/AbstractMemStateSnapshot.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemCheckpointStreamFactory.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemFoldingState.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemListState.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemReducingState.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemValueState.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/SerializedStateHandle.java rename flink-runtime/src/main/java/org/apache/flink/runtime/{instance => util}/Hardware.java (73%) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializableArrayList.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializableHashMap.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializableHashSet.java rename flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/{StateStorageHelper.java => RetrievableStateStorageHelper.java} (81%) create mode 100644 flink-runtime/src/main/scala/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala rename flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/{SavepointV0SerializerTest.java => SavepointV1SerializerTest.java} (88%) rename flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/{SavepointV0Test.java => SavepointV1Test.java} (69%) delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/LocalInputSplitsTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/HighAvailabilityModeTest.java delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyEnvironment.java.orig create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/state/KeyGroupRangeOffsetTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/state/KeyGroupRangeTest.java rename flink-runtime/src/test/java/org/apache/flink/runtime/state/{ => filesystem}/FsCheckpointStateOutputStreamTest.java (76%) rename flink-runtime/src/test/java/org/apache/flink/runtime/{instance => util}/HardwareTest.java (96%) rename flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/{CassandraConnectorTest.java => CassandraConnectorITCase.java} (84%) create mode 100644 flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BasePathBucketer.java create mode 100644 flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/Bucketer.java create mode 100644 flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java create mode 100644 flink-streaming-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/DateTimeBucketer.java create mode 100644 flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkFaultTolerance2ITCase.java create mode 100644 flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkFaultToleranceITCase.java create mode 100644 flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkTest.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkITCase.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkITCase.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/TypeUtil.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowSerializationSchema.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowSerializationSchemaTest.java create mode 100644 flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/ConfigurableStreamPartitioner.java rename flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/partitioner/{HashPartitioner.java => KeyGroupStreamPartitioner.java} (59%) delete mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskState.java delete mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskStateList.java rename flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/{HashPartitionerTest.java => KeyGroupStreamPartitionerTest.java} (71%) create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java create mode 100644 flink-test-utils-parent/flink-test-utils-junit/src/test/java/org/apache/flink/core/testutils/OneShotLatchTest.java create mode 100644 flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java create mode 100644 flink-tests/src/test/java/org/apache/flink/test/clients/examples/JobRetrievalITCase.java delete mode 100644 flink-tests/src/test/scala/org/apache/flink/api/scala/operators/translation/DistinctTranslationTest.scala diff --git a/.travis.yml b/.travis.yml index b8b39f12acaea..e15673e692ae6 100644 --- a/.travis.yml +++ b/.travis.yml @@ -44,23 +44,15 @@ matrix: git: depth: 100 -notifications: - webhooks: - urls: - - https://webhooks.gitter.im/e/d70a7e674cb9354c77b2 - on_success: always # options: [always|never|change] default: always - on_failure: always # options: [always|never|change] default: always - slack: - secure: iYjxJn8OkCRslJ30/PcE+EbMiqfKwsvUJiVUEQAEXqCEwZg+wYDsN0ilPQQT0zU16mYWKoMTx71zrOZpjirGq7ww0XZ0wAfXDjgmTxX/DaEdp87uNgTRdQzLV7mQouMKZni28eoa08Rb2NIoLLQ39q7uCu0W/p7vAD2e9xHlBBE= env: global: # Global variable to avoid hanging travis builds when downloading cache archives. - MALLOC_ARENA_MAX=2 # Build artifacts like logs (variables for apache/flink repo) - - secure: "Fm3NK28qN8yLtpJl4VI58biBECpOodMYbYXPVWwa62R7jkhHl2U1s4Xa5ujEgNIDcsUsY66z0V4pU0Es0XLNOY2ajlaFOHTmngzFIXul1r4vuNy0H8okEBjs9Ks0TOWYrE6ndAv1J4/oUsRtehayrriaehn31emXL9c4RSKgaiQ=" - - secure: "CGcWDpoPLKVPVxFCa+rh5svyrSy7tWTsydsFuLlw5BH5QR57FWH0P5ZBZ31MPppoNNpKEp1V5PBxOH0dUAx8SVNWQFNCsQrOwVpTnTlyl3Cd1udj2hahbB3l+IGf0+O3v2vv6blYm6vJb98NqzZknjdIefDDBfu52ndJy1UqHQw=" - - secure: "J4IJ7ZG5X+x/2K00kCpj6N/j3wEc6vG59KdDFVZp1WnKH8H0cty2wujZvDhBV+krbqja2MHhXQt/2mDjqm7pkdk1YElDOWsx909aw29wUdDN4yOsxFekIa5jMCrcQxbwzDRal6JmAzCakk51qIEgCYuAKquT0N+oETmnOhmcQe0=" + - secure: "c8AY4ucfq3eWpw1fzFqIoXg0B2JyBYFPruje6OJNN+eYZ/TEkXgoFXTXBYvx0Ovuy6T+nxokPyx+s+wFphVssEkJMhWZk7tYuWkOxM/ZeZ1tZpkrCUgeb2jFpmV0dbfOTeTW9ZSSSXUWCVIHfdDwm0BAoabsEwG2WcPZvnO9/js=" + - secure: "Y1VnJbGPSC2trnV0RMN1NQtYQd97/WiFGuqHsoN3G778rPyX2NN9lPg9ZkWp4SZQrJewIR+te4TWgpmckDhMSxHFjQWlj6NBGdC9wrg13Tgll1Lh5ypg7QWhlMcob32K6xWmFaDYKf0RFx5PHnlKAZN4o9EyFHZoZXanoY/PS4w=" + - secure: "Hl4fDGRUaV1YG8tWKamOZMgbmhy/NuzYRhyJI9arFkhoY5WD2waOEb+jIuEYiS6mNqgjed/Wimurpab2J5eIrHjeWZspqks0ROdCtlZCVXbXjsnado5bFOYXrrb7X3SPhm+0O99uKXdYkPyCn/WQ9Zj00Gz8urap05IzCT2JXjg=" before_script: - "gem install --no-document --version 0.8.9 faraday " diff --git a/LICENSE b/LICENSE index 07df05f2488c0..f7699dadc699d 100644 --- a/LICENSE +++ b/LICENSE @@ -225,7 +225,7 @@ The Apache Flink project bundles the following files under the MIT License: - dagre v0.7.4 (https://github.com/cpettitt/dagre) - Copyright (c) 2012-2014 Chris Pettitt - dagre-d3 v0.4.17 (https://github.com/cpettitt/dagre-d3) - Copyright (c) 2013 Chris Pettitt - EvEmitter v1.0.2 (https://github.com/metafizzy/ev-emitter) - Copyright (C) 2016 David DeSandro - - Font Awesome (code) v4.5.0 (http://fontawesome.io) - Copyright (c) 2014 Dave Gandy + - Font Awesome (code) v4.5.0, v4.6.3 (http://fontawesome.io) - Copyright (c) 2014 Dave Gandy - graphlib v1.0.7 (https://github.com/cpettitt/graphlib) - Copyright (c) 2012-2014 Chris Pettitt - imagesloaded v4.1.0 (https://github.com/desandro/imagesloaded) - Copyright (C) 2016 David DeSandro - JQuery v2.2.0 (http://jquery.com/) - Copyright 2014 jQuery Foundation and other contributors @@ -300,7 +300,8 @@ The Apache Flink project bundles the following fonts under the Open Font License (OFT) - http://scripts.sil.org/OFL - Font Awesome (http://fortawesome.github.io/Font-Awesome/) - Created by Dave Gandy - -> fonts in "flink-runtime-web/web-dashboard/assets/fonts" + -> fonts in "flink-runtime-web/web-dashboard/web/fonts" + -> fonts in "docs/page/font-awesome/fonts" ----------------------------------------------------------------------- The ISC License diff --git a/docs/README.md b/docs/README.md index 52dfad32a82d8..879c33bbc2e75 100644 --- a/docs/README.md +++ b/docs/README.md @@ -109,43 +109,19 @@ These will be replaced by a info or warning label. You can change the text of th ### Documentation -#### Top Navigation +#### Navigation -You can modify the top-level navigation in two places. You can either edit the `_includes/navbar.html` file or add tags to your page frontmatter (recommended). +The navigation on the left side of the docs is automatically generated when building the docs. You can modify the markup in `_include/sidenav.html`. - # Top-level navigation - top-nav-group: apis - top-nav-pos: 2 - top-nav-title: Batch Guide (DataSet API) +The structure of the navigation is determined by the front matter of all pages. The fields used to determine the structure are: -This adds the page to the group `apis` (via `top-nav-group`) at position `2` (via `top-nav-pos`). Furthermore, it specifies a custom title for the navigation via `top-nav-title`. If this field is missing, the regular page title (via `title`) will be used. If no position is specified, the element will be added to the end of the group. If no group is specified, the page will not show up. +- `nav-id` => ID of this page. Other pages can use this ID as their parent ID. +- `nav-parent_id` => ID of the parent. This page will be listed under the page with id `nav-parent_id`. -Currently, there are groups `quickstart`, `setup`, `deployment`, `apis`, `libs`, and `internals`. +Level 0 is made up of all pages, which have nav-parent_id set to `root`. There is no limitation on how many levels you can nest. -#### Sub Navigation +The `title` of the page is used as the default link text. You can override this via `nav-title`. The relative position per navigational level is determined by `nav-pos`. -A sub navigation is shown if the field `sub-nav-group` is specified. A sub navigation groups all pages with the same `sub-nav-group`. Check out the streaming or batch guide as an example. +If you have a page with sub pages, the link target will be used to expand the sub level navigation. If you want to actually add a link to the page as well, you can add the `nav-show_overview: true` field to the front matter. This will then add an `Overview` sub page to the expanded list. - # Sub-level navigation - sub-nav-group: batch - sub-nav-id: dataset_api - sub-nav-pos: 1 - sub-nav-title: DataSet API - -The fields work similar to their `top-nav-*` counterparts. - -In addition, you can specify a hierarchy via `sub-nav-id` and `sub-nav-parent`: - - # Sub-level navigation - sub-nav-group: batch - sub-nav-parent: dataset_api - sub-nav-pos: 1 - sub-nav-title: Transformations - -This will show the `Transformations` page under the `DataSet API` page. The `sub-nav-parent` field has to have a matching `sub-nav-id`. - -#### Breadcrumbs - -Pages with sub navigations can use breadcrumbs like `Batch Guide > Libraries > Machine Learning > Optimization`. - -The breadcrumbs for the last page are generated from the front matter. For the a sub navigation root to appear (like `Batch Guide` in the example above), you have to specify the `sub-nav-group-title`. This field designates a group page as the root. +The nesting is also used for the breadcrumbs like `Application Development > Libraries > Machine Learning > Optimization`. diff --git a/docs/_config.yml b/docs/_config.yml index d9bb57eedd001..700d28944c2e0 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -29,6 +29,7 @@ version: "1.2-SNAPSHOT" version_hadoop1: "1.2-hadoop1-SNAPSHOT" version_short: "1.2" # Used for the top navbar w/o snapshot suffix +is_snapshot_version: true # This suffix is appended to the Scala-dependent Maven artifact names scala_version_suffix: "_2.10" @@ -40,6 +41,16 @@ jira_url: "https://issues.apache.org/jira/browse/FLINK" github_url: "https://github.com/apache/flink" download_url: "http://flink.apache.org/downloads.html" +# Flag whether this is the latest stable version or not. If not, a warning +# will be printed pointing to the docs of the latest stable version. +is_latest: true +is_stable: false +latest_stable_url: http://ci.apache.org/projects/flink/flink-docs-release-1.1 + +previous_docs: + 1.1: http://ci.apache.org/projects/flink/flink-docs-release-1.1 + 1.0: http://ci.apache.org/projects/flink/flink-docs-release-1.0 + #------------------------------------------------------------------------------ # BUILD CONFIG #------------------------------------------------------------------------------ @@ -47,14 +58,16 @@ download_url: "http://flink.apache.org/downloads.html" # to change anything here. #------------------------------------------------------------------------------ +# Used in some documents to initialize arrays. Don't delete. +array: [] + defaults: - scope: path: "" values: layout: plain - top-nav-pos: 99999 # Move to end - sub-nav-pos: 99999 # Move to end + nav-pos: 99999 # Move to end if no pos specified markdown: KramdownPygments highlighter: pygments diff --git a/docs/_includes/navbar.html b/docs/_includes/navbar.html deleted file mode 100644 index 5821a469e58a6..0000000000000 --- a/docs/_includes/navbar.html +++ /dev/null @@ -1,117 +0,0 @@ - -{% capture quickstart %}{{site.baseurl}}/quickstart{% endcapture %} -{% capture setup %}{{site.baseurl}}/setup{% endcapture %} -{% capture apis %}{{site.baseurl}}/apis{% endcapture %} -{% capture libs %}{{site.baseurl}}/libs{% endcapture %} -{% capture internals %}{{site.baseurl}}/internals{% endcapture %} - -

diff --git a/docs/_includes/sidenav.html b/docs/_includes/sidenav.html new file mode 100644 index 0000000000000..b56bcf2414d54 --- /dev/null +++ b/docs/_includes/sidenav.html @@ -0,0 +1,149 @@ + + +{% comment %} +============================================================================== +Extract the active nav IDs. +============================================================================== +{% endcomment %} + +{% assign active_nav_ids = site.array %} +{% assign parent_id = page.nav-parent_id %} + +{% for i in (1..10) %} + {% if parent_id %} + {% assign active_nav_ids = active_nav_ids | push: parent_id %} + {% assign current = (site.pages | where: "nav-id" , parent_id | sort: "nav-pos") %} + {% if current.size > 0 %} + {% assign parent_id = current[0].nav-parent_id %} + {% else %} + {% break %} + {% endif %} + {% else %} + {% break %} + {% endif %} +{% endfor %} + +{% comment %} +============================================================================== +Build the nested list from nav-id and nav-parent_id relations. +============================================================================== +This builds a nested list from all pages. The fields used to determine the +structure are: + +- 'nav-id' => ID of this page. Other pages can use this ID as their + parent ID. +- 'nav-parent_id' => ID of the parent. This page will be listed under + the page with id 'nav-parent_id'. + +Level 0 is made up of all pages, which have nav-parent_id set to 'root'. + +The 'title' of the page is used as the default link text. You can +override this via 'nav-title'. The relative position per navigational +level is determined by 'nav-pos'. +{% endcomment %} + +{% assign elementsPosStack = site.array %} +{% assign posStack = site.array %} + +{% assign elements = site.array %} +{% assign children = (site.pages | where: "nav-parent_id" , "root" | sort: "nav-pos") %} +{% if children.size > 0 %} + {% assign elements = elements | push: children %} +{% endif %} + +{% assign elementsPos = 0 %} +{% assign pos = 0 %} + + +
    +{% for i in (1..10000) %} + {% if pos >= elements[elementsPos].size %} + {% if elementsPos == 0 %} + {% break %} + {% else %} + {% assign elementsPos = elementsPosStack | last %} + {% assign pos = posStack | last %} +
+ {% assign elementsPosStack = elementsPosStack | pop %} + {% assign posStack = posStack | pop %} + {% endif %} + {% else %} + {% assign this = elements[elementsPos][pos] %} + + {% if this.url == page.url %} + {% assign active = true %} + {% elsif this.nav-id and active_nav_ids contains this.nav-id %} + {% assign active = true %} + {% else %} + {% assign active = false %} + {% endif %} + + {% capture title %}{% if this.nav-title %}{{ this.nav-title }}{% else %}{{ this.title }}{% endif %}{% endcapture %} + {% capture target %}"{{ site.baseurl }}{{ this.url }}"{% if active %} class="active"{% endif %}{% endcapture %} + {% capture overview_target %}"{{ site.baseurl }}{{ this.url }}"{% if this.url == page.url %} class="active"{% endif %}{% endcapture %} + + {% assign pos = pos | plus: 1 %} + {% if this.nav-id %} + {% assign children = (site.pages | where: "nav-parent_id" , this.nav-id | sort: "nav-pos") %} + {% if children.size > 0 %} + {% capture collapse_target %}"#collapse-{{ i }}" data-toggle="collapse"{% if active %} class="active"{% endif %}{% endcapture %} + {% capture expand %}{% unless active %} {% endunless %}{% endcapture %} +
  • {{ title }}{{ expand }}
      + {% if this.nav-show_overview %}
    • Overview
    • {% endif %} + {% assign elements = elements | push: children %} + {% assign elementsPosStack = elementsPosStack | push: elementsPos %} + {% assign posStack = posStack | push: pos %} + + {% assign elementsPos = elements.size | minus: 1 %} + {% assign pos = 0 %} + {% else %} +
    • {{ title }}
    • + {% endif %} + {% else %} +
    • {{ title }}
    • + {% endif %} + {% endif %} +{% endfor %} +
    • +
    • Project Page
    • +
    + + + +
    + +
    diff --git a/docs/_layouts/base.html b/docs/_layouts/base.html index 690d2af73da30..21065ec262d5f 100644 --- a/docs/_layouts/base.html +++ b/docs/_layouts/base.html @@ -32,6 +32,7 @@ + {% if page.mathjax %}