From e251573d524f6a6a11c7821ba93fd155d9beb770 Mon Sep 17 00:00:00 2001 From: Arun Mahadevan Date: Mon, 19 Sep 2016 23:50:16 +0530 Subject: [PATCH 01/29] [STORM-1961] Stream api for storm core use cases The initial version of unified stream api for expressing streaming computation pipelines over the storm core spouts and bolts. Right now this provides at-least once guarantees and addresses only the storm core use cases. For high level design see - https://issues.apache.org/jira/secure/attachment/12827547/UnifiedStreamapiforStorm.pdf A few examples have been added which should give a basic idea on how to use the apis. More examples and detailed documentation will be added as followup tasks. --- .../storm/starter/streams/BranchExample.java | 72 +++ .../streams/GroupByKeyAndWindowExample.java | 109 ++++ .../storm/starter/streams/JoinExample.java | 110 ++++ .../starter/streams/StateQueryExample.java | 112 ++++ .../starter/streams/StatefulWordCount.java | 85 +++ .../starter/streams/WindowedWordCount.java | 90 +++ .../starter/streams/WordCountToBolt.java | 111 ++++ .../jvm/org/apache/storm/streams/Edge.java | 41 ++ .../apache/storm/streams/GroupingInfo.java | 84 +++ .../jvm/org/apache/storm/streams/Node.java | 129 ++++ .../jvm/org/apache/storm/streams/Pair.java | 78 +++ .../org/apache/storm/streams/PairStream.java | 241 +++++++ .../apache/storm/streams/PartitionNode.java | 42 ++ .../apache/storm/streams/ProcessorBolt.java | 70 +++ .../storm/streams/ProcessorBoltDelegate.java | 285 +++++++++ .../apache/storm/streams/ProcessorNode.java | 81 +++ .../apache/storm/streams/RefCountedTuple.java | 64 ++ .../org/apache/storm/streams/SinkNode.java | 44 ++ .../org/apache/storm/streams/SpoutNode.java | 48 ++ .../storm/streams/StatefulProcessorBolt.java | 116 ++++ .../jvm/org/apache/storm/streams/Stream.java | 393 ++++++++++++ .../org/apache/storm/streams/StreamBolt.java | 38 ++ .../apache/storm/streams/StreamBuilder.java | 591 ++++++++++++++++++ .../org/apache/storm/streams/StreamState.java | 43 ++ .../org/apache/storm/streams/StreamUtil.java | 62 ++ .../storm/streams/StreamsEdgeFactory.java | 29 + .../jvm/org/apache/storm/streams/Tuple3.java | 49 ++ .../org/apache/storm/streams/UniqueIdGen.java | 56 ++ .../org/apache/storm/streams/WindowNode.java | 38 ++ .../storm/streams/WindowedProcessorBolt.java | 137 ++++ .../storm/streams/operations/Aggregator.java | 42 ++ .../storm/streams/operations/Consumer.java | 32 + .../streams/operations/FlatMapFunction.java | 27 + .../storm/streams/operations/Function.java | 34 + .../streams/operations/IdentityFunction.java | 31 + .../storm/streams/operations/Operation.java | 26 + .../operations/PairFlatMapFunction.java | 30 + .../streams/operations/PairFunction.java | 30 + .../streams/operations/PairValueJoiner.java | 40 ++ .../storm/streams/operations/Predicate.java | 33 + .../streams/operations/PrintConsumer.java | 30 + .../storm/streams/operations/Reducer.java | 35 ++ .../storm/streams/operations/ValueJoiner.java | 36 ++ .../streams/operations/aggregators/Count.java | 37 ++ .../streams/operations/aggregators/Sum.java | 35 ++ .../operations/mappers/PairValueMapper.java | 51 ++ .../operations/mappers/TupleValueMapper.java | 30 + .../operations/mappers/ValueMapper.java | 45 ++ .../operations/mappers/ValuesMapper.java | 48 ++ .../processors/AggregateByKeyProcessor.java | 54 ++ .../processors/AggregateProcessor.java | 45 ++ .../streams/processors/BaseProcessor.java | 107 ++++ .../streams/processors/BatchProcessor.java | 25 + .../streams/processors/BranchProcessor.java | 41 ++ .../processors/ChainedProcessorContext.java | 66 ++ .../processors/EmittingProcessorContext.java | 170 +++++ .../streams/processors/FilterProcessor.java | 35 ++ .../streams/processors/FlatMapProcessor.java | 35 ++ .../processors/FlatMapValuesProcessor.java | 36 ++ .../streams/processors/ForEachProcessor.java | 33 + .../ForwardingProcessorContext.java | 102 +++ .../streams/processors/JoinProcessor.java | 112 ++++ .../streams/processors/MapProcessor.java | 33 + .../processors/MapValuesProcessor.java | 34 + .../streams/processors/PeekProcessor.java | 34 + .../storm/streams/processors/Processor.java | 51 ++ .../streams/processors/ProcessorContext.java | 59 ++ .../processors/ReduceByKeyProcessor.java | 52 ++ .../streams/processors/ReduceProcessor.java | 41 ++ .../processors/StateQueryProcessor.java | 48 ++ .../streams/processors/StatefulProcessor.java | 36 ++ .../processors/UpdateStateByKeyProcessor.java | 49 ++ .../storm/streams/windowing/BaseWindow.java | 64 ++ .../streams/windowing/SlidingWindows.java | 151 +++++ .../streams/windowing/TumblingWindows.java | 119 ++++ .../storm/streams/windowing/Window.java | 70 +++ .../StatefulWindowedBoltExecutor.java | 4 +- .../storm/topology/WindowedBoltExecutor.java | 4 +- .../storm/topology/base/BaseWindowedBolt.java | 37 +- .../AbstractTridentWindowManager.java | 2 +- .../storm/windowing/CountEvictionPolicy.java | 8 +- .../storm/windowing/EvictionPolicy.java | 7 + .../storm/windowing/TimeEvictionPolicy.java | 17 +- .../storm/windowing/TupleWindowImpl.java | 11 + .../WatermarkCountEvictionPolicy.java | 10 +- .../WatermarkTimeEvictionPolicy.java | 4 +- .../org/apache/storm/windowing/Window.java | 7 + .../windowing/WindowLifecycleListener.java | 3 +- .../apache/storm/windowing/WindowManager.java | 2 +- .../storm/streams/ProcessorBoltTest.java | 165 +++++ .../streams/StatefulProcessorBoltTest.java | 100 +++ .../storm/streams/StreamBuilderTest.java | 219 +++++++ .../streams/WindowedProcessorBoltTest.java | 110 ++++ .../storm/windowing/WindowManagerTest.java | 2 +- 94 files changed, 6505 insertions(+), 29 deletions(-) create mode 100644 examples/storm-starter/src/jvm/org/apache/storm/starter/streams/BranchExample.java create mode 100644 examples/storm-starter/src/jvm/org/apache/storm/starter/streams/GroupByKeyAndWindowExample.java create mode 100644 examples/storm-starter/src/jvm/org/apache/storm/starter/streams/JoinExample.java create mode 100644 examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java create mode 100644 examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StatefulWordCount.java create mode 100644 examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WindowedWordCount.java create mode 100644 examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WordCountToBolt.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/Edge.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/GroupingInfo.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/Node.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/Pair.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/PairStream.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/PartitionNode.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/ProcessorBolt.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/ProcessorBoltDelegate.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/ProcessorNode.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/RefCountedTuple.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/SinkNode.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/SpoutNode.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/StatefulProcessorBolt.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/Stream.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/StreamBolt.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/StreamState.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/StreamUtil.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/StreamsEdgeFactory.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/Tuple3.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/UniqueIdGen.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/WindowNode.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/WindowedProcessorBolt.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/Aggregator.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/Consumer.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/FlatMapFunction.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/Function.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/IdentityFunction.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/Operation.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/PairFlatMapFunction.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/PairFunction.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/PairValueJoiner.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/Predicate.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/PrintConsumer.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/Reducer.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/ValueJoiner.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Count.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Sum.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/mappers/PairValueMapper.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/mappers/TupleValueMapper.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/mappers/ValueMapper.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/mappers/ValuesMapper.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/AggregateByKeyProcessor.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/AggregateProcessor.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/BaseProcessor.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/BatchProcessor.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/BranchProcessor.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/ChainedProcessorContext.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/EmittingProcessorContext.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/FilterProcessor.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/FlatMapProcessor.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/FlatMapValuesProcessor.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/ForEachProcessor.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/ForwardingProcessorContext.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/JoinProcessor.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/MapProcessor.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/MapValuesProcessor.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/PeekProcessor.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/Processor.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/ProcessorContext.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/ReduceByKeyProcessor.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/ReduceProcessor.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/StateQueryProcessor.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/StatefulProcessor.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/UpdateStateByKeyProcessor.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/windowing/BaseWindow.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/windowing/SlidingWindows.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/windowing/TumblingWindows.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/windowing/Window.java create mode 100644 storm-core/test/jvm/org/apache/storm/streams/ProcessorBoltTest.java create mode 100644 storm-core/test/jvm/org/apache/storm/streams/StatefulProcessorBoltTest.java create mode 100644 storm-core/test/jvm/org/apache/storm/streams/StreamBuilderTest.java create mode 100644 storm-core/test/jvm/org/apache/storm/streams/WindowedProcessorBoltTest.java diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/BranchExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/BranchExample.java new file mode 100644 index 00000000000..f5400a5cbbc --- /dev/null +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/BranchExample.java @@ -0,0 +1,72 @@ +/** + * 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.storm.starter.streams; + +import org.apache.storm.Config; +import org.apache.storm.LocalCluster; +import org.apache.storm.StormSubmitter; +import org.apache.storm.starter.spout.RandomIntegerSpout; +import org.apache.storm.streams.Stream; +import org.apache.storm.streams.StreamBuilder; +import org.apache.storm.streams.operations.Predicate; +import org.apache.storm.streams.operations.mappers.ValueMapper; +import org.apache.storm.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An example that demonstrates the usage of {@link Stream#branch(Predicate[])} to split a stream + * into multiple branches based on predicates. + */ +public class BranchExample { + private static final Logger LOG = LoggerFactory.getLogger(BranchExample.class); + + @SuppressWarnings("unchecked") + public static void main(String[] args) throws Exception { + StreamBuilder builder = new StreamBuilder(); + Stream[] evenAndOdd = builder + /* + * Create a stream of random numbers from a spout that + * emits random integers by extracting the tuple value at index 0. + */ + .newStream(new RandomIntegerSpout(), new ValueMapper(0)) + /* + * Split the stream of numbers into streams of + * even and odd numbers. The first stream contains even + * and the second contains odd numbers. + */ + .branch(x -> (x % 2) == 0, + x -> (x % 2) == 1); + + evenAndOdd[0].forEach(x -> LOG.info("EVEN> " + x)); + evenAndOdd[1].forEach(x -> LOG.info("ODD > " + x)); + + Config config = new Config(); + if (args.length > 0) { + config.setNumWorkers(1); + StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build()); + } else { + LocalCluster cluster = new LocalCluster(); + cluster.submitTopology("test", config, builder.build()); + Utils.sleep(60000); + cluster.killTopology("test"); + cluster.shutdown(); + } + } + +} diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/GroupByKeyAndWindowExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/GroupByKeyAndWindowExample.java new file mode 100644 index 00000000000..6b505bdc95e --- /dev/null +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/GroupByKeyAndWindowExample.java @@ -0,0 +1,109 @@ +/** + * 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.storm.starter.streams; + +import org.apache.storm.Config; +import org.apache.storm.LocalCluster; +import org.apache.storm.StormSubmitter; +import org.apache.storm.spout.SpoutOutputCollector; +import org.apache.storm.streams.PairStream; +import org.apache.storm.streams.StreamBuilder; +import org.apache.storm.streams.operations.Reducer; +import org.apache.storm.streams.operations.mappers.PairValueMapper; +import org.apache.storm.streams.windowing.SlidingWindows; +import org.apache.storm.streams.windowing.Window; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.topology.base.BaseRichSpout; +import org.apache.storm.topology.base.BaseWindowedBolt.Count; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Values; +import org.apache.storm.utils.Utils; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +/** + * An example that shows the usage of {@link PairStream#groupByKeyAndWindow(Window)} + * and {@link PairStream#reduceByKeyAndWindow(Reducer, Window)} + */ +public class GroupByKeyAndWindowExample { + public static void main(String[] args) throws Exception { + StreamBuilder builder = new StreamBuilder(); + + // a stream of stock quotes + builder.newStream(new StockQuotes(), new PairValueMapper(0, 1)) + /* + * The elements having the same key within the window will be grouped + * together and the corresponding values will be merged. + */ + .groupByKeyAndWindow(SlidingWindows.of(Count.of(6), Count.of(3))) + .print(); + + // a stream of stock quotes + builder.newStream(new StockQuotes(), new PairValueMapper(0, 1)) + /* + * The elements having the same key within the window will be grouped + * together and their values will be reduced using the given reduce function. + */ + .reduceByKeyAndWindow((x, y) -> (x + y) / 2.0, SlidingWindows.of(Count.of(6), Count.of(3))) + .print(); + + Config config = new Config(); + if (args.length > 0) { + config.setNumWorkers(1); + StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build()); + } else { + LocalCluster cluster = new LocalCluster(); + config.put(Config.TOPOLOGY_STATE_PROVIDER, "org.apache.storm.redis.state.RedisKeyValueStateProvider"); + cluster.submitTopology("test", config, builder.build()); + Utils.sleep(60000); + cluster.killTopology("test"); + cluster.shutdown(); + } + } + + private static class StockQuotes extends BaseRichSpout { + private final List> values = Arrays.asList( + Arrays.asList(new Values("AAPL", 100.0), new Values("GOOG", 780.0), new Values("FB", 125.0)), + Arrays.asList(new Values("AAPL", 105.0), new Values("GOOG", 790.0), new Values("FB", 130.0)) + ); + private SpoutOutputCollector collector; + private int index = 0; + + @Override + public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { + this.collector = collector; + } + + @Override + public void nextTuple() { + Utils.sleep(5000); + for (Values v : values.get(index)) { + collector.emit(v); + } + index = (index + 1) % values.size(); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("key", "val")); + } + } +} diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/JoinExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/JoinExample.java new file mode 100644 index 00000000000..0b15615f2bb --- /dev/null +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/JoinExample.java @@ -0,0 +1,110 @@ +/** + * 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.storm.starter.streams; + +import org.apache.storm.Config; +import org.apache.storm.LocalCluster; +import org.apache.storm.StormSubmitter; +import org.apache.storm.spout.SpoutOutputCollector; +import org.apache.storm.streams.PairStream; +import org.apache.storm.streams.StreamBuilder; +import org.apache.storm.streams.operations.Function; +import org.apache.storm.streams.operations.mappers.PairValueMapper; +import org.apache.storm.streams.windowing.TumblingWindows; +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 java.util.Map; + +import static org.apache.storm.topology.base.BaseWindowedBolt.Duration; + +/** + * An example that demonstrates the usage of {@link PairStream#join(PairStream)} to join + * multiple streams. + */ +public class JoinExample { + public static void main(String[] args) throws Exception { + StreamBuilder builder = new StreamBuilder(); + // a stream of (number, square) pairs + PairStream squares = builder + .newStream(new NumberSpout(x -> x * x), + new PairValueMapper<>(0, 1)); + // a stream of (number, cube) pairs + PairStream cubes = builder + .newStream(new NumberSpout(x -> x * x * x), + new PairValueMapper<>(0, 1)); + + // create a windowed stream of five seconds duration + squares.window(TumblingWindows.of(Duration.seconds(5))) + /* + * Join the squares and the cubes stream within the window. + * The values in the squares stream having the same key as that + * of the cubes stream within the window will be joined together. + */ + .join(cubes) + /** + * The results should be of the form (number, (square, cube)) + */ + .print(); + + Config config = new Config(); + if (args.length > 0) { + config.setNumWorkers(1); + StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build()); + } else { + LocalCluster cluster = new LocalCluster(); + cluster.submitTopology("test", config, builder.build()); + Utils.sleep(60000); + cluster.killTopology("test"); + cluster.shutdown(); + } + + } + + private static class NumberSpout extends BaseRichSpout { + private final Function function; + private SpoutOutputCollector collector; + private int i = 1; + + NumberSpout(Function function) { + this.function = function; + } + + @Override + public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { + this.collector = collector; + } + + @Override + public void nextTuple() { + Utils.sleep(990); + collector.emit(new Values(i, function.apply(i))); + i++; + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("key", "val")); + } + } + +} diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java new file mode 100644 index 00000000000..e76dd3c0fc6 --- /dev/null +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java @@ -0,0 +1,112 @@ +/** + * 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.storm.starter.streams; + +import org.apache.storm.Config; +import org.apache.storm.LocalCluster; +import org.apache.storm.StormSubmitter; +import org.apache.storm.spout.SpoutOutputCollector; +import org.apache.storm.streams.Pair; +import org.apache.storm.streams.Stream; +import org.apache.storm.streams.StreamBuilder; +import org.apache.storm.streams.StreamState; +import org.apache.storm.streams.operations.aggregators.Count; +import org.apache.storm.streams.operations.mappers.ValueMapper; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.testing.TestWordSpout; +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 java.util.Map; + +/** + * An example that uses {@link Stream#stateQuery(StreamState)} to query the state + *

+ * You should start a local redis instance before running the 'storm jar' command. By default + * the connection will be attempted at localhost:6379. The default + * RedisKeyValueStateProvider parameters can be overridden in conf/storm.yaml, for e.g. + *

+ *

+ * topology.state.provider.config: '{"keyClass":"...", "valueClass":"...",
+ *  "keySerializerClass":"...", "valueSerializerClass":"...",
+ *  "jedisPoolConfig":{"host":"localhost", "port":6379,
+ *  "timeout":2000, "database":0, "password":"xyz"}}'
+ * 
+ */ +public class StateQueryExample { + public static void main(String[] args) throws Exception { + StreamBuilder builder = new StreamBuilder(); + StreamState ss = builder.newStream(new TestWordSpout(), new ValueMapper(0)) + .mapToPair(w -> Pair.of(w, 1)) + .groupByKey() + .updateStateByKey(new Count<>()); + + /* + * A stream of words emitted by the QuerySpout is used as + * the keys to query the state. + */ + builder.newStream(new QuerySpout(), new ValueMapper(0)) + /* + * Queries the state and emits the + * matching (key, value) as results. The stream state returned + * by the updateStateByKey is passed as the argument to stateQuery. + */ + .stateQuery(ss).print(); + + Config config = new Config(); + // use redis based state store for persistence + config.put(Config.TOPOLOGY_STATE_PROVIDER, "org.apache.storm.redis.state.RedisKeyValueStateProvider"); + + if (args.length > 0) { + config.setNumWorkers(1); + StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build()); + } else { + LocalCluster cluster = new LocalCluster(); + cluster.submitTopology("test", config, builder.build()); + Utils.sleep(60000); + cluster.killTopology("test"); + cluster.shutdown(); + } + } + + private static class QuerySpout extends BaseRichSpout { + private SpoutOutputCollector collector; + private final String[] words = {"nathan", "mike"}; + + @Override + public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { + this.collector = collector; + } + + @Override + public void nextTuple() { + Utils.sleep(2000); + for (String word : words) { + collector.emit(new Values(word)); + } + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declare(new Fields("word")); + } + } +} diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StatefulWordCount.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StatefulWordCount.java new file mode 100644 index 00000000000..f6ae6b07496 --- /dev/null +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StatefulWordCount.java @@ -0,0 +1,85 @@ +/** + * 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.storm.starter.streams; + +import org.apache.storm.Config; +import org.apache.storm.LocalCluster; +import org.apache.storm.StormSubmitter; +import org.apache.storm.streams.Pair; +import org.apache.storm.streams.PairStream; +import org.apache.storm.streams.StreamBuilder; +import org.apache.storm.streams.operations.Aggregator; +import org.apache.storm.streams.operations.aggregators.Count; +import org.apache.storm.streams.operations.mappers.ValueMapper; +import org.apache.storm.testing.TestWordSpout; +import org.apache.storm.utils.Utils; + +/** + * A stateful word count that uses {@link PairStream#updateStateByKey(Aggregator)} to + * save the counts in a key value state. This example uses Redis state store. + *

+ * You should start a local redis instance before running the 'storm jar' command. By default + * the connection will be attempted at localhost:6379. The default + * RedisKeyValueStateProvider parameters can be overridden in conf/storm.yaml, for e.g. + *

+ *

+ * topology.state.provider.config: '{"keyClass":"...", "valueClass":"...",
+ *  "keySerializerClass":"...", "valueSerializerClass":"...",
+ *  "jedisPoolConfig":{"host":"localhost", "port":6379,
+ *  "timeout":2000, "database":0, "password":"xyz"}}'
+ * 
+ */ +public class StatefulWordCount { + public static void main(String[] args) throws Exception { + StreamBuilder builder = new StreamBuilder(); + // a stream of words + builder.newStream(new TestWordSpout(), new ValueMapper(0)) + /* + * create a stream of (word, 1) pairs + */ + .mapToPair(w -> Pair.of(w, 1)) + /* + * group by the word + */ + .groupByKey() + /* + * update the word counts in the state + */ + .updateStateByKey(new Count<>()) + /* + * convert the state back to a stream and print the results + */ + .toPairStream() + .print(); + + Config config = new Config(); + // use redis based state store for persistence + config.put(Config.TOPOLOGY_STATE_PROVIDER, "org.apache.storm.redis.state.RedisKeyValueStateProvider"); + + if (args.length > 0) { + config.setNumWorkers(1); + StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build()); + } else { + LocalCluster cluster = new LocalCluster(); + cluster.submitTopology("test", config, builder.build()); + Utils.sleep(60000); + cluster.killTopology("test"); + cluster.shutdown(); + } + } +} diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WindowedWordCount.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WindowedWordCount.java new file mode 100644 index 00000000000..c6e2f4a1491 --- /dev/null +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WindowedWordCount.java @@ -0,0 +1,90 @@ +/** + * 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.storm.starter.streams; + +import org.apache.storm.Config; +import org.apache.storm.LocalCluster; +import org.apache.storm.StormSubmitter; +import org.apache.storm.starter.spout.RandomSentenceSpout; +import org.apache.storm.streams.Pair; +import org.apache.storm.streams.StreamBuilder; +import org.apache.storm.streams.operations.aggregators.Count; +import org.apache.storm.streams.operations.mappers.ValueMapper; +import org.apache.storm.streams.windowing.TumblingWindows; +import org.apache.storm.utils.Utils; + +import java.util.Arrays; + +import static org.apache.storm.topology.base.BaseWindowedBolt.Duration; + +/** + * A windowed word count example + */ +public class WindowedWordCount { + public static void main(String[] args) throws Exception { + StreamBuilder builder = new StreamBuilder(); + // A stream of random sentences + builder.newStream(new RandomSentenceSpout(), new ValueMapper(0)) + /* + * Increase the parallelism of this stream. Further operations + * on this stream will execute at this level of parallelism. + */ + .repartition(2) + /* + * split the sentences to words + */ + .flatMap(s -> Arrays.asList(s.split(" "))) + /* + * create a stream of (word, 1) pairs + */ + .mapToPair(w -> Pair.of(w, 1)) + /* + * group by word so that the same words end up in the same partition + */ + .groupByKey() + /* + * a two seconds tumbling window + */ + .window(TumblingWindows.of(Duration.seconds(2))) + /* + * compute the word counts in the last two second window + */ + .aggregateByKey(new Count<>()) + /* + * emit the count for the words that occurred + * at-least five times in the last two seconds + */ + .filter(x -> x.getSecond() >= 5) + /* + * print the results to stdout + */ + .print(); + + Config config = new Config(); + if (args.length > 0) { + config.setNumWorkers(1); + StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build()); + } else { + LocalCluster cluster = new LocalCluster(); + cluster.submitTopology("test", config, builder.build()); + Utils.sleep(60000); + cluster.killTopology("test"); + cluster.shutdown(); + } + } +} diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WordCountToBolt.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WordCountToBolt.java new file mode 100644 index 00000000000..a7116963d37 --- /dev/null +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WordCountToBolt.java @@ -0,0 +1,111 @@ +/** + * 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.storm.starter.streams; + +import org.apache.storm.Config; +import org.apache.storm.LocalCluster; +import org.apache.storm.StormSubmitter; +import org.apache.storm.redis.bolt.RedisStoreBolt; +import org.apache.storm.redis.common.config.JedisPoolConfig; +import org.apache.storm.redis.common.mapper.RedisDataTypeDescription; +import org.apache.storm.redis.common.mapper.RedisStoreMapper; +import org.apache.storm.streams.Pair; +import org.apache.storm.streams.StreamBuilder; +import org.apache.storm.streams.operations.aggregators.Count; +import org.apache.storm.streams.operations.mappers.ValueMapper; +import org.apache.storm.testing.TestWordSpout; +import org.apache.storm.topology.IRichBolt; +import org.apache.storm.tuple.ITuple; +import org.apache.storm.utils.Utils; + +/** + * An example that computes word counts and finally emits the results to an + * external bolt (sink) + */ +public class WordCountToBolt { + public static void main(String[] args) throws Exception { + StreamBuilder builder = new StreamBuilder(); + + // Redis config parameters for the RedisStoreBolt + JedisPoolConfig poolConfig = new JedisPoolConfig.Builder() + .setHost("127.0.0.1").setPort(6379).build(); + // Storm tuple to redis key-value mapper + RedisStoreMapper storeMapper = new WordCountStoreMapper(); + // The redis bolt (sink) + IRichBolt redisStoreBolt = new RedisStoreBolt(poolConfig, storeMapper); + + // A stream of words + builder.newStream(new TestWordSpout(), new ValueMapper(0)) + /* + * create a stream of (word, 1) pairs + */ + .mapToPair(w -> Pair.of(w, 1)) + /* + * group by key (word) + */ + .groupByKey() + /* + * aggregate the count + */ + .aggregateByKey(new Count<>()) + /* + * The result of aggregation is forwarded to + * the RedisStoreBolt. The forwarded tuple is a + * key-value pair of (word, count) with ("key", "value") + * being the field names. + */ + .to(redisStoreBolt); + + Config config = new Config(); + if (args.length > 0) { + config.setNumWorkers(1); + StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build()); + } else { + LocalCluster cluster = new LocalCluster(); + cluster.submitTopology("test", config, builder.build()); + Utils.sleep(60000); + cluster.killTopology("test"); + cluster.shutdown(); + } + } + + // Maps a storm tuple to redis key and value + private static class WordCountStoreMapper implements RedisStoreMapper { + private final RedisDataTypeDescription description; + private final String hashKey = "wordCount"; + + WordCountStoreMapper() { + description = new RedisDataTypeDescription(RedisDataTypeDescription.RedisDataType.HASH, hashKey); + } + + @Override + public RedisDataTypeDescription getDataTypeDescription() { + return description; + } + + @Override + public String getKeyFromTuple(ITuple tuple) { + return tuple.getStringByField("key"); + } + + @Override + public String getValueFromTuple(ITuple tuple) { + return String.valueOf(tuple.getLongByField("value")); + } + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/Edge.java b/storm-core/src/jvm/org/apache/storm/streams/Edge.java new file mode 100644 index 00000000000..9b13562717e --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/Edge.java @@ -0,0 +1,41 @@ +/** + * 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.storm.streams; + +import java.io.Serializable; + +/** + * An edge connects source and target nodes + */ +class Edge implements Serializable { + private final Node source; + private final Node target; + + Edge(Node source, Node target) { + this.source = source; + this.target = target; + } + + public Node getSource() { + return source; + } + + public Node getTarget() { + return target; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/GroupingInfo.java b/storm-core/src/jvm/org/apache/storm/streams/GroupingInfo.java new file mode 100644 index 00000000000..81def4b0580 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/GroupingInfo.java @@ -0,0 +1,84 @@ +/** + * 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.storm.streams; + +import org.apache.storm.topology.BoltDeclarer; +import org.apache.storm.tuple.Fields; + +import java.io.Serializable; + +abstract class GroupingInfo implements Serializable { + private final Fields fields; + + private GroupingInfo() { + this(null); + } + + private GroupingInfo(Fields fields) { + this.fields = fields; + } + + public abstract void declareGrouping(BoltDeclarer declarer, String componentId, String streamId, Fields fields); + + public static GroupingInfo shuffle() { + return new GroupingInfo() { + @Override + public void declareGrouping(BoltDeclarer declarer, String componentId, String streamId, Fields fields) { + declarer.shuffleGrouping(componentId, streamId); + } + }; + } + + public static GroupingInfo fields(Fields fields) { + return new GroupingInfo(fields) { + @Override + public void declareGrouping(BoltDeclarer declarer, String componentId, String streamId, Fields fields) { + declarer.fieldsGrouping(componentId, streamId, fields); + } + }; + } + + public static GroupingInfo global() { + return new GroupingInfo() { + @Override + public void declareGrouping(BoltDeclarer declarer, String componentId, String streamId, Fields fields) { + declarer.globalGrouping(componentId, streamId); + } + }; + } + + public static GroupingInfo all() { + return new GroupingInfo() { + @Override + public void declareGrouping(BoltDeclarer declarer, String componentId, String streamId, Fields fields) { + declarer.allGrouping(componentId, streamId); + } + }; + } + + public Fields getFields() { + return fields; + } + + @Override + public String toString() { + return "GroupingInfo{" + + "fields=" + fields + + '}'; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/Node.java b/storm-core/src/jvm/org/apache/storm/streams/Node.java new file mode 100644 index 00000000000..f9de390a9c9 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/Node.java @@ -0,0 +1,129 @@ +/** + * 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.storm.streams; + +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.Multimap; +import com.google.common.collect.Multimaps; +import org.apache.storm.generated.StreamInfo; +import org.apache.storm.topology.IComponent; +import org.apache.storm.topology.OutputFieldsGetter; +import org.apache.storm.tuple.Fields; + +import java.io.Serializable; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +/** + * Base class for a Node which form the vertices of the topology DAG. + */ +abstract class Node implements Serializable { + private final Set outputStreams; + protected final Fields outputFields; + protected String componentId; + protected int parallelism; + // the parent streams that this node subscribes to + private final Multimap parentStreams = ArrayListMultimap.create(); + + Node(Set outputStreams, Fields outputFields, String componentId, int parallelism) { + this.outputStreams = new HashSet<>(outputStreams); + this.outputFields = outputFields; + this.componentId = componentId; + this.parallelism = parallelism; + } + + Node(String outputStream, Fields outputFields, String componentId, int parallelism) { + this(Collections.singleton(outputStream), outputFields, componentId, parallelism); + } + + Node(String outputStream, Fields outputFields, String componentId) { + this(outputStream, outputFields, componentId, 1); + } + + Node(String outputStream, Fields outputFields) { + this(outputStream, outputFields, null); + } + + public Fields getOutputFields() { + return outputFields; + } + + String getComponentId() { + return componentId; + } + + void setComponentId(String componentId) { + this.componentId = componentId; + } + + Integer getParallelism() { + return parallelism; + } + + void setParallelism(int parallelism) { + this.parallelism = parallelism; + } + + void addParentStream(Node parent, String streamId) { + parentStreams.put(parent, streamId); + } + + void removeParentStreams(Node parent) { + parentStreams.removeAll(parent); + } + + Set getOutputStreams() { + return Collections.unmodifiableSet(outputStreams); + } + + Collection getParentStreams(Node parent) { + return parentStreams.get(parent); + } + + Set getParents(String stream) { + Multimap rev = Multimaps.invertFrom(parentStreams, ArrayListMultimap.create()); + return new HashSet<>(rev.get(stream)); + } + + void addOutputStream(String streamId) { + outputStreams.add(streamId); + } + + static Fields getOutputFields(IComponent component, String streamId) { + OutputFieldsGetter getter = new OutputFieldsGetter(); + component.declareOutputFields(getter); + Map fieldsDeclaration = getter.getFieldsDeclaration(); + if ((fieldsDeclaration != null) && fieldsDeclaration.containsKey(streamId)) { + return new Fields(fieldsDeclaration.get(streamId).get_output_fields()); + } + return new Fields(); + } + + @Override + public String toString() { + return "Node{" + + "outputStreams='" + outputStreams + '\'' + + ", outputFields=" + outputFields + + ", componentId='" + componentId + '\'' + + ", parallelism=" + parallelism + + '}'; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/Pair.java b/storm-core/src/jvm/org/apache/storm/streams/Pair.java new file mode 100644 index 00000000000..00443598510 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/Pair.java @@ -0,0 +1,78 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.streams; + +import java.io.Serializable; + +/** + * A pair of values. + * + * @param the type of the first value + * @param the type of the second value + */ +public final class Pair implements Serializable { + private final T1 first; + private final T2 second; + + /** + * Constructs a new pair of values + * + * @param first the first value + * @param second the second value + */ + private Pair(T1 first, T2 second) { + this.first = first; + this.second = second; + } + + /** + * Returns the first value in a pair. + * + * @return the first value + */ + public T1 getFirst() { + return first; + } + + /** + * Returns the second value in a pair. + * + * @return the second value + */ + public T2 getSecond() { + return second; + } + + /** + * Constructs a new pair of values. + * + * @param first the first value + * @param second the second value + * @param the type of the first value + * @param the type of the second value + * @return a new pair of values + */ + public static Pair of(T1 first, T2 second) { + return new Pair<>(first, second); + } + + @Override + public String toString() { + return "(" + first + ", " + second + ')'; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/PairStream.java b/storm-core/src/jvm/org/apache/storm/streams/PairStream.java new file mode 100644 index 00000000000..2d18b30536f --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/PairStream.java @@ -0,0 +1,241 @@ +/** + * 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.storm.streams; + +import org.apache.storm.Config; +import org.apache.storm.streams.operations.Aggregator; +import org.apache.storm.streams.operations.Consumer; +import org.apache.storm.streams.operations.FlatMapFunction; +import org.apache.storm.streams.operations.Function; +import org.apache.storm.streams.operations.PairValueJoiner; +import org.apache.storm.streams.operations.Predicate; +import org.apache.storm.streams.operations.Reducer; +import org.apache.storm.streams.operations.ValueJoiner; +import org.apache.storm.streams.processors.AggregateByKeyProcessor; +import org.apache.storm.streams.processors.FlatMapValuesProcessor; +import org.apache.storm.streams.processors.JoinProcessor; +import org.apache.storm.streams.processors.MapValuesProcessor; +import org.apache.storm.streams.processors.ReduceByKeyProcessor; +import org.apache.storm.streams.processors.UpdateStateByKeyProcessor; +import org.apache.storm.streams.windowing.Window; +import org.apache.storm.tuple.Fields; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +/** + * Represents a stream of key-value pairs. + * + * @param the key type + * @param the value type + */ +public class PairStream extends Stream> { + + PairStream(StreamBuilder topology, Node node) { + super(topology, node); + } + + /** + * Returns a new stream by applying a {@link Function} to the value of each key-value pairs in + * this stream. + * + * @param function the mapping function + * @param the result type + * @return the new stream + */ + public PairStream mapValues(Function function) { + return new PairStream<>(streamBuilder, addProcessorNode(new MapValuesProcessor<>(function), KEY_VALUE)); + } + + /** + * Return a new stream by applying a {@link FlatMapFunction} function to the value of each key-value pairs in + * this stream. + * + * @param function the flatmap function + * @param the result type + * @return the new stream + */ + public PairStream flatMapValues(FlatMapFunction function) { + return new PairStream<>(streamBuilder, addProcessorNode(new FlatMapValuesProcessor<>(function), KEY_VALUE)); + } + + /** + * Aggregates the values for each key of this stream using the given {@link Aggregator}. + * + * @param aggregator the aggregator + * @param the result type + * @return the new stream + */ + public PairStream aggregateByKey(Aggregator aggregator) { + return new PairStream<>(streamBuilder, addProcessorNode(new AggregateByKeyProcessor<>(aggregator), KEY_VALUE)); + } + + /** + * Performs a reduction on the values for each key of this stream by repeatedly applying the reducer. + * + * @param reducer the reducer + * @return the new stream + */ + public PairStream reduceByKey(Reducer reducer) { + return new PairStream<>(streamBuilder, addProcessorNode(new ReduceByKeyProcessor<>(reducer), KEY_VALUE)); + } + + /** + * Returns a new stream where the values are grouped by the keys. + * + * @return the new stream + */ + public PairStream groupByKey() { + return partitionBy(KEY); + } + + /** + * Returns a new stream where the values are grouped by keys and the given window. + * The values that arrive within a window having the same key will be merged together and returned + * as an Iterable of values mapped to the key. + * + * @param window the window configuration + * @return the new stream + */ + public PairStream> groupByKeyAndWindow(Window window) { + return groupByKey().window(window).aggregateByKey(new MergeValues<>()); + } + + /** + * Returns a new stream where the values that arrive within a window + * having the same key will be reduced by repeatedly applying the reducer. + * + * @param reducer the reducer + * @param window the window configuration + * @return the new stream + */ + public PairStream reduceByKeyAndWindow(Reducer reducer, Window window) { + return groupByKey().window(window).reduceByKey(reducer); + } + + /** + * {@inheritDoc} + */ + @Override + public PairStream peek(Consumer> action) { + return toPairStream(super.peek(action)); + } + + /** + * Join the values of this stream with the values having the same key from the other stream. + *

+ * Note: The parallelism and windowing parameters (if windowed) of this stream is carried forward to the joined stream. + *

+ * + * @param otherStream the other stream + * @param the type of the values in the other stream + * @return the new stream + */ + public PairStream> join(PairStream otherStream) { + return join(otherStream, new PairValueJoiner<>()); + } + + /** + * Join the values of this stream with the values having the same key from the other stream. + *

+ * Note: The parallelism and windowing parameters (if windowed) of this stream is carried forward to the joined stream. + *

+ * + * @param otherStream the other stream + * @param valueJoiner the {@link ValueJoiner} + * @param the type of the values resulting from the join + * @param the type of the values in the other stream + * @return the new stream + */ + public PairStream join(PairStream otherStream, + ValueJoiner valueJoiner) { + String leftStream = stream; + String rightStream = otherStream.stream; + Node joinNode = addProcessorNode(new JoinProcessor<>(leftStream, rightStream, valueJoiner), KEY_VALUE); + addNode(otherStream.getNode(), joinNode, joinNode.getParallelism()); + return new PairStream<>(streamBuilder, joinNode); + } + + /** + * {@inheritDoc} + */ + @Override + public PairStream window(Window window) { + return toPairStream(super.window(window)); + } + + /** + * {@inheritDoc} + */ + @Override + public PairStream repartition(int parallelism) { + return toPairStream(super.repartition(parallelism)); + } + + /** + * {@inheritDoc} + */ + @Override + @SuppressWarnings("unchecked") + public PairStream[] branch(Predicate>... predicates) { + List> pairStreams = new ArrayList<>(); + for (Stream> stream : super.branch(predicates)) { + pairStreams.add(toPairStream(stream)); + } + return pairStreams.toArray(new PairStream[pairStreams.size()]); + } + + /** + * Update the state by applying the given aggregator to the previous state of the + * key and the new value for the key. This internally uses {@link org.apache.storm.topology.IStatefulBolt} + * to save the state. Use {@link Config#TOPOLOGY_STATE_PROVIDER} to choose the state implementation. + * + * @param aggregator the aggregator + * @param the result type + * @return the {@link StreamState} which can be used to query the state + */ + public StreamState updateStateByKey(Aggregator aggregator) { + return new StreamState<>( + new PairStream<>(streamBuilder, addProcessorNode(new UpdateStateByKeyProcessor<>(aggregator), KEY_VALUE))); + } + + private PairStream partitionBy(Fields fields) { + return new PairStream<>( + streamBuilder, + addNode(new PartitionNode(stream, node.getOutputFields(), GroupingInfo.fields(fields)))); + } + + private PairStream toPairStream(Stream> stream) { + return new PairStream<>(stream.streamBuilder, stream.node); + } + + // used internally to merge values in groupByKeyAndWindow + private static class MergeValues implements Aggregator> { + @Override + public ArrayList init() { + return new ArrayList<>(); + } + + @Override + public ArrayList apply(V value, ArrayList aggregate) { + aggregate.add(value); + return aggregate; + } + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/PartitionNode.java b/storm-core/src/jvm/org/apache/storm/streams/PartitionNode.java new file mode 100644 index 00000000000..ca92deff2a9 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/PartitionNode.java @@ -0,0 +1,42 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.streams; + +import org.apache.storm.tuple.Fields; + +/** + * Node that holds partitioning/grouping information. + * This is used for operations like groupBy (fields grouping), global + * aggregate/reduce (global grouping), state query (all grouping). + */ +class PartitionNode extends Node { + private final GroupingInfo groupingInfo; + + PartitionNode(String outputStream, Fields outputFields, GroupingInfo groupingInfo) { + super(outputStream, outputFields); + this.groupingInfo = groupingInfo; + } + + PartitionNode(String outputStream, Fields outputFields) { + this(outputStream, outputFields, GroupingInfo.shuffle()); + } + + GroupingInfo getGroupingInfo() { + return groupingInfo; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/ProcessorBolt.java b/storm-core/src/jvm/org/apache/storm/streams/ProcessorBolt.java new file mode 100644 index 00000000000..f1163ca7d08 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/ProcessorBolt.java @@ -0,0 +1,70 @@ +/** + * 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.storm.streams; + +import com.google.common.collect.Multimap; +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.Tuple; +import org.jgrapht.DirectedGraph; + +import java.util.List; +import java.util.Map; + +/** + * Stream bolt that executes the different processors (except windowed and stateful operations) + */ +class ProcessorBolt extends BaseRichBolt implements StreamBolt { + private final ProcessorBoltDelegate delegate; + + ProcessorBolt(String id, DirectedGraph graph, List nodes) { + delegate = new ProcessorBoltDelegate(id, graph, nodes); + } + + @Override + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + delegate.prepare(stormConf, context, collector); + } + + @Override + public void execute(Tuple input) { + delegate.processAndAck(input); + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + delegate.declareOutputFields(declarer); + } + + + @Override + public void setTimestampField(String fieldName) { + delegate.setTimestampField(fieldName); + } + + @Override + public String getId() { + return delegate.getId(); + } + + void setStreamToInitialProcessors(Multimap streamToInitialProcessors) { + delegate.setStreamToInitialProcessors(streamToInitialProcessors); + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/ProcessorBoltDelegate.java b/storm-core/src/jvm/org/apache/storm/streams/ProcessorBoltDelegate.java new file mode 100644 index 00000000000..5bc6fff1a5f --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/ProcessorBoltDelegate.java @@ -0,0 +1,285 @@ +/** + * 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.storm.streams; + +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.Multimap; +import org.apache.storm.streams.processors.ChainedProcessorContext; +import org.apache.storm.streams.processors.EmittingProcessorContext; +import org.apache.storm.streams.processors.ForwardingProcessorContext; +import org.apache.storm.streams.processors.Processor; +import org.apache.storm.streams.processors.ProcessorContext; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Tuple; +import org.jgrapht.DirectedGraph; +import org.jgrapht.graph.DirectedSubgraph; +import org.jgrapht.traverse.TopologicalOrderIterator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +class ProcessorBoltDelegate implements Serializable { + private static final Logger LOG = LoggerFactory.getLogger(ProcessorBoltDelegate.class); + private final String id; + private final DirectedGraph graph; + private final List nodes; + private Map stormConf; + private TopologyContext topologyContext; + private OutputCollector outputCollector; + private final List outgoingProcessors = new ArrayList<>(); + private final Set emittingProcessorContexts = new HashSet<>(); + private final Map> punctuationState = new HashMap<>(); + private Multimap streamToInitialProcessors; + private String timestampField; + + ProcessorBoltDelegate(String id, DirectedGraph graph, List nodes) { + this.id = id; + this.graph = graph; + this.nodes = new ArrayList<>(nodes); + } + + String getId() { + return id; + } + + void addNodes(Collection nodes) { + this.nodes.addAll(nodes); + } + + List getNodes() { + return Collections.unmodifiableList(nodes); + } + + void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + this.stormConf = stormConf; + topologyContext = context; + outputCollector = collector; + DirectedSubgraph subgraph = new DirectedSubgraph<>(graph, new HashSet<>(nodes), null); + TopologicalOrderIterator it = new TopologicalOrderIterator<>(subgraph); + while (it.hasNext()) { + Node node = it.next(); + if (!(node instanceof ProcessorNode)) { + throw new IllegalStateException("Not a processor node " + node); + } + ProcessorNode processorNode = (ProcessorNode) node; + List children = StreamUtil.getChildren(subgraph, processorNode); + ProcessorContext processorContext; + if (children.isEmpty()) { + processorContext = createEmittingContext(processorNode); + } else { + Multimap streamToChildren = ArrayListMultimap.create(); + for (ProcessorNode child : children) { + for (String stream : child.getParentStreams(processorNode)) { + streamToChildren.put(stream, child); + } + } + ForwardingProcessorContext forwardingContext = new ForwardingProcessorContext(processorNode, streamToChildren); + if (hasOutgoingChild(processorNode, new HashSet<>(children))) { + processorContext = new ChainedProcessorContext(processorNode, forwardingContext, createEmittingContext(processorNode)); + } else { + processorContext = forwardingContext; + } + } + processorNode.initProcessorContext(processorContext); + } + if (timestampField != null) { + for (EmittingProcessorContext ctx : emittingProcessorContexts) { + ctx.setTimestampField(timestampField); + } + } + } + + void declareOutputFields(OutputFieldsDeclarer declarer) { + for (ProcessorNode node : nodes) { + for (String stream : node.getOutputStreams()) { + if (timestampField == null) { + declarer.declareStream(stream, node.getOutputFields()); + } else { + List fields = new ArrayList<>(); + fields.addAll(node.getOutputFields().toList()); + fields.add(timestampField); + declarer.declareStream(stream, new Fields(fields)); + } + } + } + } + + void setAnchor(RefCountedTuple tuple) { + for (EmittingProcessorContext ctx : emittingProcessorContexts) { + ctx.setAnchor(tuple); + } + } + + Pair getValueAndStream(Tuple input) { + Object value; + String stream; + // if tuple arrives from a spout, it can be passed as is + // otherwise the value is in the first field of the tuple + if (input.getSourceComponent().startsWith("spout")) { + value = input; + stream = input.getSourceGlobalStreamId().get_componentId() + input.getSourceGlobalStreamId().get_streamId(); + } else if (isPair(input)) { + value = Pair.of(input.getValue(0), input.getValue(1)); + stream = input.getSourceStreamId(); + } else { + value = input.getValue(0); + stream = input.getSourceStreamId(); + } + return Pair.of(value, stream); + } + + void processAndAck(Tuple input) { + RefCountedTuple refCountedTuple = new RefCountedTuple(input); + setAnchor(refCountedTuple); + if (isEventTimestamp()) { + setEventTimestamp(input.getLongByField(getTimestampField())); + } + Pair valueAndStream = getValueAndStream(input); + process(valueAndStream.getFirst(), valueAndStream.getSecond()); + ack(refCountedTuple); + } + + void process(Object value, String sourceStreamId) { + LOG.debug("Process value {}, sourceStreamId {}", value, sourceStreamId); + Collection initialProcessors = streamToInitialProcessors.get(sourceStreamId); + for (ProcessorNode processorNode : initialProcessors) { + Processor processor = processorNode.getProcessor(); + if (StreamUtil.isPunctuation(value)) { + if (shouldPunctuate(processorNode, sourceStreamId)) { + processor.punctuate(null); + clearPunctuationState(processorNode); + } + } else { + processor.execute(value, sourceStreamId); + } + } + } + + void setStreamToInitialProcessors(Multimap streamToInitialProcessors) { + this.streamToInitialProcessors = streamToInitialProcessors; + } + + void addStreamToInitialProcessors(Multimap streamToInitialProcessors) { + this.streamToInitialProcessors.putAll(streamToInitialProcessors); + } + + Set getInitialStreams() { + return streamToInitialProcessors.keySet(); + } + + void setTimestampField(String fieldName) { + timestampField = fieldName; + } + + boolean isEventTimestamp() { + return timestampField != null; + } + + void setEventTimestamp(long timestamp) { + for (EmittingProcessorContext ctx : emittingProcessorContexts) { + ctx.setEventTimestamp(timestamp); + } + } + + private String getTimestampField() { + return timestampField; + } + + // if there are no windowed/batched processors, we would ack immediately + private void ack(RefCountedTuple tuple) { + if (tuple.shouldAck()) { + LOG.debug("ACKing tuple {}", tuple); + outputCollector.ack(tuple.tuple()); + tuple.setAcked(); + } + } + + private ProcessorContext createEmittingContext(ProcessorNode processorNode) { + List emittingContexts = new ArrayList<>(); + for (String stream : processorNode.getOutputStreams()) { + EmittingProcessorContext emittingContext = new EmittingProcessorContext(processorNode, outputCollector, stream); + if (StreamUtil.isSinkStream(stream)) { + emittingContext.setEmitPunctuation(false); + } + emittingContexts.add(emittingContext); + } + emittingProcessorContexts.addAll(emittingContexts); + outgoingProcessors.add(processorNode); + return new ChainedProcessorContext(processorNode, emittingContexts); + } + + private boolean hasOutgoingChild(ProcessorNode processorNode, Set boltChildren) { + for (Node child : getChildNodes(processorNode)) { + if ((child instanceof ProcessorNode && !boltChildren.contains(child)) + || child instanceof SinkNode) { + return true; + } + } + return false; + } + + private Set getChildNodes(Node node) { + Set children = new HashSet<>(); + for (Node child : StreamUtil.getChildren(graph, node)) { + if (child instanceof WindowNode || child instanceof PartitionNode) { + children.addAll(getChildNodes(child)); + } else { + children.add(child); + } + } + return children; + } + + // if we received punctuation from all parent windowed streams + private boolean shouldPunctuate(ProcessorNode processorNode, String sourceStreamId) { + if (processorNode.getWindowedParentStreams().size() <= 1) { + return true; + } + Set receivedStreams = punctuationState.get(processorNode); + if (receivedStreams == null) { + receivedStreams = new HashSet<>(); + punctuationState.put(processorNode, receivedStreams); + } + receivedStreams.add(sourceStreamId); + return receivedStreams.equals(processorNode.getWindowedParentStreams()); + } + + private void clearPunctuationState(ProcessorNode processorNode) { + Set state = punctuationState.get(processorNode); + if (state != null) { + state.clear(); + } + } + + private boolean isPair(Tuple input) { + return input.size() == (timestampField == null ? 2 : 3); + } + +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/ProcessorNode.java b/storm-core/src/jvm/org/apache/storm/streams/ProcessorNode.java new file mode 100644 index 00000000000..4771f4f8b3f --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/ProcessorNode.java @@ -0,0 +1,81 @@ +/** + * 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.storm.streams; + +import org.apache.storm.streams.processors.BatchProcessor; +import org.apache.storm.streams.processors.Processor; +import org.apache.storm.streams.processors.ProcessorContext; +import org.apache.storm.tuple.Fields; + +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + +/** + * Node that wraps a processor in the stream. + */ +public class ProcessorNode extends Node { + private final Processor processor; + private final boolean isBatch; + private boolean windowed; + // Windowed parent streams + private Set windowedParentStreams = Collections.emptySet(); + + public ProcessorNode(Processor processor, String outputStream, Fields outputFields) { + super(outputStream, outputFields); + this.isBatch = processor instanceof BatchProcessor; + this.processor = processor; + } + + public Processor getProcessor() { + return processor; + } + + public boolean isWindowed() { + return windowed; + } + + public boolean isBatch() { + return isBatch; + } + + public void setWindowed(boolean windowed) { + this.windowed = windowed; + } + + public Set getWindowedParentStreams() { + return Collections.unmodifiableSet(windowedParentStreams); + } + + void initProcessorContext(ProcessorContext context) { + processor.init(context); + } + + void setWindowedParentStreams(Set windowedParentStreams) { + this.windowedParentStreams = new HashSet<>(windowedParentStreams); + } + + @Override + public String toString() { + return "ProcessorNode{" + + "processor=" + processor + + ", windowed=" + windowed + + ", windowedParentStreams=" + windowedParentStreams + + "} " + super.toString(); + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/RefCountedTuple.java b/storm-core/src/jvm/org/apache/storm/streams/RefCountedTuple.java new file mode 100644 index 00000000000..e58bcc5999a --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/RefCountedTuple.java @@ -0,0 +1,64 @@ +/** + * 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.storm.streams; + +import org.apache.storm.tuple.Tuple; + +/** + * Provides reference counting of tuples. Used when operations that operate + * on a batch of tuples are involved (e.g. aggregation, join etc). + * The input tuples are acked once the result is emitted downstream. + */ +public class RefCountedTuple { + private int count = 0; + private final Tuple tuple; + private boolean acked; + + RefCountedTuple(Tuple tuple) { + this.tuple = tuple; + this.acked = false; + } + + public boolean shouldAck() { + return count == 0 && !acked; + } + + public void increment() { + ++count; + } + + public void decrement() { + --count; + } + + public Tuple tuple() { + return tuple; + } + + public void setAcked() { + acked = true; + } + + @Override + public String toString() { + return "RefCountedTuple{" + + "count=" + count + + ", tuple=" + tuple + + '}'; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/SinkNode.java b/storm-core/src/jvm/org/apache/storm/streams/SinkNode.java new file mode 100644 index 00000000000..d95ab6b525f --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/SinkNode.java @@ -0,0 +1,44 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.streams; + +import org.apache.storm.topology.IBasicBolt; +import org.apache.storm.topology.IComponent; +import org.apache.storm.topology.IRichBolt; +import org.apache.storm.utils.Utils; + +/** + * Sink node holds IRich or IBasic bolts that are passed + * via the {@code Stream#to()} api. + */ +class SinkNode extends Node { + private final IComponent bolt; + + SinkNode(IComponent bolt) { + super(Utils.DEFAULT_STREAM_ID, getOutputFields(bolt, Utils.DEFAULT_STREAM_ID)); + if (bolt instanceof IRichBolt || bolt instanceof IBasicBolt) { + this.bolt = bolt; + } else { + throw new IllegalArgumentException("Should be an IRichBolt or IBasicBolt"); + } + } + + IComponent getBolt() { + return bolt; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/SpoutNode.java b/storm-core/src/jvm/org/apache/storm/streams/SpoutNode.java new file mode 100644 index 00000000000..47845142a6e --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/SpoutNode.java @@ -0,0 +1,48 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.streams; + +import org.apache.storm.topology.IRichSpout; +import org.apache.storm.tuple.Fields; +import org.apache.storm.utils.Utils; + +/** + * A spout node wraps an {@link IRichSpout}. + */ +class SpoutNode extends Node { + private final IRichSpout spout; + + SpoutNode(IRichSpout spout) { + super(Utils.DEFAULT_STREAM_ID, getOutputFields(spout, Utils.DEFAULT_STREAM_ID)); + if (outputFields.size() == 0) { + throw new IllegalArgumentException("Spout " + spout + " does not declare any fields" + + "for the stream '" + Utils.DEFAULT_STREAM_ID + "'"); + } + this.spout = spout; + } + + IRichSpout getSpout() { + return spout; + } + + @Override + void addOutputStream(String streamId) { + throw new UnsupportedOperationException("Cannot add output streams to a spout node"); + } + +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/StatefulProcessorBolt.java b/storm-core/src/jvm/org/apache/storm/streams/StatefulProcessorBolt.java new file mode 100644 index 00000000000..c1236585802 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/StatefulProcessorBolt.java @@ -0,0 +1,116 @@ +/** + * 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.storm.streams; + +import com.google.common.collect.Multimap; +import org.apache.storm.state.KeyValueState; +import org.apache.storm.streams.processors.StatefulProcessor; +import org.apache.storm.streams.processors.UpdateStateByKeyProcessor; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.topology.base.BaseStatefulBolt; +import org.apache.storm.tuple.Tuple; +import org.jgrapht.DirectedGraph; + +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * Stream bolt that executes stateful operations like update state and state query. + */ +class StatefulProcessorBolt extends BaseStatefulBolt> implements StreamBolt { + private final ProcessorBoltDelegate delegate; + // can be UpdateStateByKey or StateQuery processors + private final Set> statefulProcessors; + + StatefulProcessorBolt(String boltId, DirectedGraph graph, List nodes) { + delegate = new ProcessorBoltDelegate(boltId, graph, nodes); + statefulProcessors = getStatefulProcessors(nodes); + } + + @Override + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + delegate.prepare(stormConf, context, collector); + } + + @Override + public void execute(Tuple input) { + delegate.processAndAck(input); + } + + @Override + public void initState(KeyValueState state) { + for (StatefulProcessor statefulProcessor : statefulProcessors) { + statefulProcessor.initState(state); + } + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + delegate.declareOutputFields(declarer); + } + + @Override + public void setTimestampField(String fieldName) { + delegate.setTimestampField(fieldName); + } + + @Override + public String getId() { + return delegate.getId(); + } + + void setStreamToInitialProcessors(Multimap streamToInitialProcessors) { + delegate.setStreamToInitialProcessors(streamToInitialProcessors); + } + + public List getNodes() { + return delegate.getNodes(); + } + + void addStreamToInitialProcessors(Multimap streamToInitialProcessors) { + delegate.addStreamToInitialProcessors(streamToInitialProcessors); + } + + void addNodes(List nodes) { + delegate.addNodes(nodes); + statefulProcessors.addAll(getStatefulProcessors(nodes)); + } + + @SuppressWarnings("unchecked") + private Set> getStatefulProcessors(List nodes) { + Set> statefulProcessors = new HashSet<>(); + int updateStateByKeyCount = 0; + for (ProcessorNode node : nodes) { + if (node.getProcessor() instanceof StatefulProcessor) { + statefulProcessors.add((StatefulProcessor) node.getProcessor()); + if (node.getProcessor() instanceof UpdateStateByKeyProcessor) { + if (++updateStateByKeyCount > 1) { + throw new IllegalArgumentException("Cannot have more than one updateStateByKey processor " + + "in a StatefulProcessorBolt"); + } + } + + } + } + return statefulProcessors; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/Stream.java b/storm-core/src/jvm/org/apache/storm/streams/Stream.java new file mode 100644 index 00000000000..e50e7a25d5d --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/Stream.java @@ -0,0 +1,393 @@ +/** + * 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.storm.streams; + +import org.apache.storm.streams.operations.Aggregator; +import org.apache.storm.streams.operations.Consumer; +import org.apache.storm.streams.operations.FlatMapFunction; +import org.apache.storm.streams.operations.Function; +import org.apache.storm.streams.operations.IdentityFunction; +import org.apache.storm.streams.operations.PairFlatMapFunction; +import org.apache.storm.streams.operations.PairFunction; +import org.apache.storm.streams.operations.Predicate; +import org.apache.storm.streams.operations.PrintConsumer; +import org.apache.storm.streams.operations.Reducer; +import org.apache.storm.streams.processors.AggregateProcessor; +import org.apache.storm.streams.processors.BranchProcessor; +import org.apache.storm.streams.processors.FilterProcessor; +import org.apache.storm.streams.processors.FlatMapProcessor; +import org.apache.storm.streams.processors.ForEachProcessor; +import org.apache.storm.streams.processors.MapProcessor; +import org.apache.storm.streams.processors.PeekProcessor; +import org.apache.storm.streams.processors.Processor; +import org.apache.storm.streams.processors.ReduceProcessor; +import org.apache.storm.streams.processors.StateQueryProcessor; +import org.apache.storm.streams.windowing.Window; +import org.apache.storm.topology.IBasicBolt; +import org.apache.storm.topology.IRichBolt; +import org.apache.storm.tuple.Fields; +import org.apache.storm.utils.Utils; + +import java.util.ArrayList; +import java.util.List; + +/** + * Represents a stream of values. + * + * @param the type of the value + */ +public class Stream { + protected static final Fields KEY = new Fields("key"); + protected static final Fields VALUE = new Fields("value"); + protected static final Fields KEY_VALUE = new Fields("key", "value"); + // the stream builder + protected final StreamBuilder streamBuilder; + // the current node + protected final Node node; + // the stream id from node's output stream(s) that this stream represents + protected final String stream; + + Stream(StreamBuilder streamBuilder, Node node) { + this(streamBuilder, node, node.getOutputStreams().iterator().next()); + } + + private Stream(StreamBuilder streamBuilder, Node node, String stream) { + this.streamBuilder = streamBuilder; + this.node = node; + this.stream = stream; + } + + /** + * Returns a stream consisting of the elements of this stream that matches the given filter. + * + * @param predicate the predicate to apply to each element to determine if it should be included + * @return the new stream + */ + public Stream filter(Predicate predicate) { + return new Stream<>(streamBuilder, addProcessorNode(new FilterProcessor<>(predicate), VALUE)); + } + + /** + * Returns a stream consisting of the result of applying the given mapping function to the values of this stream. + * + * @param function a mapping function to be applied to each value in this stream. + * @return the new stream + */ + public Stream map(Function function) { + return new Stream<>(streamBuilder, addProcessorNode(new MapProcessor<>(function), VALUE)); + } + + /** + * Returns a stream of key-value pairs by applying a {@link PairFunction} on each value of this stream. + * + * @param function the mapping function to be applied to each value in this stream + * @param the key type + * @param the value type + * @return the new stream of key-value pairs + */ + public PairStream mapToPair(PairFunction function) { + return new PairStream<>(streamBuilder, addProcessorNode(new MapProcessor<>(function), KEY_VALUE)); + } + + /** + * Returns a stream consisting of the results of replacing each value of this stream with the contents + * produced by applying the provided mapping function to each value. This has the effect of applying + * a one-to-many transformation to the values of the stream, and then flattening the resulting elements + * into a new stream. + * + * @param function a mapping function to be applied to each value in this stream which produces new values. + * @return the new stream + */ + public Stream flatMap(FlatMapFunction function) { + return new Stream<>(streamBuilder, addProcessorNode(new FlatMapProcessor<>(function), VALUE)); + } + + /** + * Returns a stream consisting of the results of replacing each value of this stream with the key-value pairs + * produced by applying the provided mapping function to each value. + * + * @param function the mapping function to be applied to each value in this stream which produces new key-value pairs. + * @param the key type + * @param the value type + * @return the new stream of key-value pairs + * @see #flatMap(FlatMapFunction) + * @see #mapToPair(PairFunction) + */ + public PairStream flatMapToPair(PairFlatMapFunction function) { + return new PairStream<>(streamBuilder, addProcessorNode(new FlatMapProcessor<>(function), KEY_VALUE)); + } + + /** + * Returns a new stream consisting of the elements that fall within the window as specified by the window parameter. + * The {@link Window} specification could be used to specify sliding or tumbling windows based on + * time duration or event count. For example, + *
+     * // time duration based sliding window
+     * stream.window(SlidingWindows.of(Duration.minutes(10), Duration.minutes(1));
+     *
+     * // count based sliding window
+     * stream.window(SlidingWindows.of(Count.(10), Count.of(2)));
+     *
+     * // time duration based tumbling window
+     * stream.window(TumblingWindows.of(Duration.seconds(10));
+     * 

+ * + * @see org.apache.storm.streams.windowing.SlidingWindows + * @see org.apache.storm.streams.windowing.TumblingWindows + * @param window the window configuration + * @return the new stream + */ + public Stream window(Window window) { + return new Stream<>(streamBuilder, addNode(new WindowNode(window, stream, node.getOutputFields()))); + } + + /** + * Performs an action for each element of this stream. + * + * @param action an action to perform on the elements + */ + public void forEach(Consumer action) { + addProcessorNode(new ForEachProcessor<>(action), new Fields()); + } + + /** + * Returns a stream consisting of the elements of this stream, additionally performing the provided action on + * each element as they are consumed from the resulting stream. + * + * @param action the action to perform on the element as they are consumed from the stream + * @return the new stream + */ + public Stream peek(Consumer action) { + return new Stream<>(streamBuilder, addProcessorNode(new PeekProcessor<>(action), node.getOutputFields())); + } + + /** + * Aggregates the values in this stream using the aggregator. This does a global aggregation, i.e. the elements + * across all the partitions are forwarded to a single task for computing the aggregate. + *

+ * If the stream is windowed, the aggregate result is emitted after each window activation and represents the + * aggregate of elements that fall within that window. + * If the stream is not windowed, the aggregate result is emitted as each new element in the stream is processed. + *

+ * + * @param aggregator the aggregator + * @param the result type + * @return the new stream + */ + public Stream aggregate(Aggregator aggregator) { + return new Stream<>(streamBuilder, global().addProcessorNode(new AggregateProcessor<>(aggregator), VALUE)); + } + + /** + * Performs a reduction on the elements of this stream, by repeatedly applying the reducer. + *

+ * If the stream is windowed, the result is emitted after each window activation and represents the + * reduction of elements that fall within that window. + * If the stream is not windowed, the result is emitted as each new element in the stream is processed. + *

+ * + * @param reducer the reducer + * @return the new stream + */ + public Stream reduce(Reducer reducer) { + return new Stream<>(streamBuilder, global().addProcessorNode(new ReduceProcessor<>(reducer), VALUE)); + } + + /** + * Returns a new stream with the given value of parallelism. Further operations on this stream + * would execute at this level of parallelism. + * + * @param parallelism the parallelism value + * @return the new stream + */ + public Stream repartition(int parallelism) { + if (parallelism < 1) { + throw new IllegalArgumentException("Parallelism should be >= 1"); + } + Node partitionNode = addNode(node, new PartitionNode(stream, node.getOutputFields()), parallelism); + return new Stream<>(streamBuilder, partitionNode); + } + + /** + * Returns an array of streams by splitting the given stream into multiple branches based on the given + * predicates. The predicates are applied in the given order to the values of this stream and the result + * is forwarded to the corresponding (index based) result stream based on the (index of) predicate that matches. + * If none of the predicates match a value, that value is dropped. + * + * @param predicates the predicates + * @return an array of result streams (branches) corresponding to the given predicates + */ + @SuppressWarnings("unchecked") + public Stream[] branch(Predicate... predicates) { + List> childStreams = new ArrayList<>(); + if (predicates.length > 0) { + BranchProcessor branchProcessor = new BranchProcessor<>(); + Node branchNode = addProcessorNode(branchProcessor, VALUE); + for (Predicate predicate : predicates) { + // create a child node (identity) per branch + ProcessorNode child = makeProcessorNode(new MapProcessor<>(new IdentityFunction<>()), node.getOutputFields()); + String branchStream = child.getOutputStreams().iterator().next() + "-branch"; + // branchStream is the parent stream that connects branch processor to this child + branchNode.addOutputStream(branchStream); + addNode(branchNode, child, branchStream); + childStreams.add(new Stream<>(streamBuilder, child)); + branchProcessor.addPredicate(predicate, branchStream); + } + } + return childStreams.toArray((Stream[]) new Stream[childStreams.size()]); + } + + /** + * Print the values in this stream. + */ + public void print() { + forEach(new PrintConsumer()); + } + + /** + * Sends the elements of this stream to a bolt. This could be used to plug in existing bolts as + * sinks in the stream, for e.g. a {@code RedisStoreBolt}. The bolt would have a parallelism of 1. + *

+ * Note: This would provide guarantees only based on what the bolt provides. + *

+ * + * @param bolt the bolt + */ + public void to(IRichBolt bolt) { + to(bolt, 1); + } + + /** + * Sends the elements of this stream to a bolt. This could be used to plug in existing bolts as + * sinks in the stream, for e.g. a {@code RedisStoreBolt}. + *

+ * Note: This would provide guarantees only based on what the bolt provides. + *

+ * + * @param bolt the bolt + * @param parallelism the parallelism of the bolt + */ + public void to(IRichBolt bolt, int parallelism) { + addSinkNode(new SinkNode(bolt), parallelism); + } + + /** + * Sends the elements of this stream to a bolt. This could be used to plug in existing bolts as + * sinks in the stream, for e.g. a {@code RedisStoreBolt}. The bolt would have a parallelism of 1. + *

+ * Note: This would provide guarantees only based on what the bolt provides. + *

+ * + * @param bolt the bolt + */ + public void to(IBasicBolt bolt) { + to(bolt, 1); + } + + /** + * Sends the elements of this stream to a bolt. This could be used to plug in existing bolts as + * sinks in the stream, for e.g. a {@code RedisStoreBolt}. + *

+ * Note: This would provide guarantees only based on what the bolt provides. + *

+ * + * @param bolt the bolt + * @param parallelism the parallelism of the bolt + */ + public void to(IBasicBolt bolt, int parallelism) { + addSinkNode(new SinkNode(bolt), parallelism); + } + + /** + * Queries the given stream state with the values in this stream as the keys. + * + * @param streamState the stream state + * @param the value type + * @return the result stream + */ + public PairStream stateQuery(StreamState streamState) { + // need all grouping for state query since the state is local + Node node = all().addProcessorNode(new StateQueryProcessor<>(streamState), KEY_VALUE); + // add 'updateState' node as parent so that state query gets processed after update state + addNode(streamState.getUpdateStateNode(), node, node.getParallelism()); + return new PairStream<>(streamBuilder, node); + } + + Node getNode() { + return node; + } + + Node addNode(Node parent, Node child, int parallelism) { + return streamBuilder.addNode(parent, child, parallelism); + } + + Node addNode(Node child) { + return addNode(this.node, child); + } + + Node addProcessorNode(Processor processor, Fields outputFields) { + return addNode(makeProcessorNode(processor, outputFields)); + } + + String getStream() { + return stream; + } + + private Node addNode(Node parent, Node child) { + return streamBuilder.addNode(parent, child); + } + + private Node addNode(Node parent, Node child, String parentStreamId) { + return streamBuilder.addNode(parent, child, parentStreamId); + } + + private Node addNode(Node child, int parallelism, String parentStreamId) { + return streamBuilder.addNode(this.node, child, parallelism, parentStreamId); + } + + private ProcessorNode makeProcessorNode(Processor processor, Fields outputFields) { + return new ProcessorNode(processor, UniqueIdGen.getInstance().getUniqueStreamId(), outputFields); + } + + private void addSinkNode(SinkNode sinkNode, int parallelism) { + String boltId = UniqueIdGen.getInstance().getUniqueBoltId(); + sinkNode.setComponentId(boltId); + sinkNode.setParallelism(parallelism); + if (node instanceof SpoutNode) { + addNode(sinkNode, parallelism, Utils.DEFAULT_STREAM_ID); + } else { + /* + * add a stream__sink stream to the current node (parent) for emitting + * just the values (no punctuation) to the bolt. + */ + String sinkStream = StreamUtil.getSinkStream(stream); + node.addOutputStream(sinkStream); + addNode(sinkNode, parallelism, sinkStream); + } + } + + private Stream global() { + Node partitionNode = addNode(new PartitionNode(stream, node.getOutputFields(), GroupingInfo.global())); + return new Stream<>(streamBuilder, partitionNode); + } + + private Stream all() { + Node partitionNode = addNode(new PartitionNode(stream, node.getOutputFields(), GroupingInfo.all())); + return new Stream<>(streamBuilder, partitionNode); + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/StreamBolt.java b/storm-core/src/jvm/org/apache/storm/streams/StreamBolt.java new file mode 100644 index 00000000000..6170ea8703d --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/StreamBolt.java @@ -0,0 +1,38 @@ +/** + * 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.storm.streams; + +/** + * Interface for bolts that executes the streaming operations via the + * processors. + */ +interface StreamBolt { + /** + * The bolt component id. This is auto generated value like (bolt-N). + * + * @return the bolt id. + */ + String getId(); + + /** + * Set the timestamp field name for event time based processing. + * + * @param fieldName the timestamp field name + */ + void setTimestampField(String fieldName); +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java b/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java new file mode 100644 index 00000000000..e19a0c63bda --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java @@ -0,0 +1,591 @@ +/** + * 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.storm.streams; + +import com.google.common.base.Function; +import com.google.common.base.Predicate; +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.Collections2; +import com.google.common.collect.HashBasedTable; +import com.google.common.collect.Multimap; +import com.google.common.collect.Table; +import org.apache.storm.generated.StormTopology; +import org.apache.storm.streams.operations.IdentityFunction; +import org.apache.storm.streams.operations.mappers.PairValueMapper; +import org.apache.storm.streams.operations.mappers.TupleValueMapper; +import org.apache.storm.streams.processors.JoinProcessor; +import org.apache.storm.streams.processors.MapProcessor; +import org.apache.storm.streams.processors.StateQueryProcessor; +import org.apache.storm.streams.processors.StatefulProcessor; +import org.apache.storm.streams.windowing.Window; +import org.apache.storm.topology.BoltDeclarer; +import org.apache.storm.topology.IBasicBolt; +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.Tuple; +import org.jgrapht.graph.DefaultDirectedGraph; +import org.jgrapht.traverse.TopologicalOrderIterator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.PriorityQueue; +import java.util.Set; + +/** + * A builder for constructing a {@link StormTopology} via storm streams api (DSL) + */ +public class StreamBuilder { + private static final Logger LOG = LoggerFactory.getLogger(StreamBuilder.class); + private final DefaultDirectedGraph graph; + private final Table nodeGroupingInfo = HashBasedTable.create(); + private final Map windowInfo = new HashMap<>(); + private final List curGroup = new ArrayList<>(); + private int statefulProcessorCount = 0; + private final Map streamBolts = new HashMap<>(); + private String timestampFieldName = null; + + /** + * Creates a new {@link StreamBuilder} + */ + public StreamBuilder() { + graph = new DefaultDirectedGraph<>(new StreamsEdgeFactory()); + } + + /** + * Creates a new {@link Stream} of tuples from the given {@link IRichSpout} + * + * @param spout the spout + * @return the new stream + */ + public Stream newStream(IRichSpout spout) { + return newStream(spout, 1); + } + + /** + * Creates a new {@link Stream} of tuples from the given {@link IRichSpout} with the given + * parallelism. + * + * @param spout the spout + * @param parallelism the parallelism of the stream + * @return the new stream + */ + public Stream newStream(IRichSpout spout, int parallelism) { + SpoutNode spoutNode = new SpoutNode(spout); + String spoutId = UniqueIdGen.getInstance().getUniqueSpoutId(); + spoutNode.setComponentId(spoutId); + spoutNode.setParallelism(parallelism); + graph.addVertex(spoutNode); + return new Stream<>(this, spoutNode); + } + + /** + * Creates a new {@link Stream} of values from the given {@link IRichSpout} by extracting field(s) + * from tuples via the supplied {@link TupleValueMapper}. + * + * @param spout the spout + * @param valueMapper the value mapper + * @param the type of values in the resultant stream + * @return the new stream + */ + public Stream newStream(IRichSpout spout, TupleValueMapper valueMapper) { + return newStream(spout).map(valueMapper); + } + + + /** + * Creates a new {@link Stream} of values from the given {@link IRichSpout} by extracting field(s) + * from tuples via the supplied {@link TupleValueMapper} with the given parallelism. + * + * @param spout the spout + * @param valueMapper the value mapper + * @param parallelism the parallelism of the stream + * @param the type of values in the resultant stream + * @return the new stream + */ + public Stream newStream(IRichSpout spout, TupleValueMapper valueMapper, int parallelism) { + return newStream(spout, parallelism).map(valueMapper); + } + + /** + * Creates a new {@link PairStream} of key-value pairs from the given {@link IRichSpout} by extracting key and + * value from tuples via the supplied {@link PairValueMapper}. + * + * @param spout the spout + * @param pairValueMapper the pair value mapper + * @param the key type + * @param the value type + * @return the new stream of key-value pairs + */ + public PairStream newStream(IRichSpout spout, PairValueMapper pairValueMapper) { + return newStream(spout).mapToPair(pairValueMapper); + } + + /** + * Creates a new {@link PairStream} of key-value pairs from the given {@link IRichSpout} by extracting key and + * value from tuples via the supplied {@link PairValueMapper} and with the given value of parallelism. + * + * @param spout the spout + * @param pairValueMapper the pair value mapper + * @param parallelism the parallelism of the stream + * @param the key type + * @param the value type + * @return the new stream of key-value pairs + */ + public PairStream newStream(IRichSpout spout, PairValueMapper pairValueMapper, int parallelism) { + return newStream(spout, parallelism).mapToPair(pairValueMapper); + } + + + /** + * Builds a new {@link StormTopology} for the computation expressed + * via the stream api. + * + * @return the storm topology + */ + public StormTopology build() { + nodeGroupingInfo.clear(); + windowInfo.clear(); + curGroup.clear(); + TopologicalOrderIterator iterator = new TopologicalOrderIterator<>(graph, queue()); + TopologyBuilder topologyBuilder = new TopologyBuilder(); + while (iterator.hasNext()) { + Node node = iterator.next(); + if (node instanceof SpoutNode) { + addSpout(topologyBuilder, (SpoutNode) node); + } else if (node instanceof ProcessorNode) { + handleProcessorNode((ProcessorNode) node, topologyBuilder); + } else if (node instanceof PartitionNode) { + updateNodeGroupingInfo((PartitionNode) node); + processCurGroup(topologyBuilder); + } else if (node instanceof WindowNode) { + updateWindowInfo((WindowNode) node); + processCurGroup(topologyBuilder); + } else if (node instanceof SinkNode) { + processCurGroup(topologyBuilder); + addSink(topologyBuilder, (SinkNode) node); + } + } + processCurGroup(topologyBuilder); + mayBeAddTsField(); + return topologyBuilder.createTopology(); + } + + Node addNode(Node parent, Node child) { + return addNode(parent, child, parent.getParallelism(), parent.getOutputStreams().iterator().next()); + } + + Node addNode(Node parent, Node child, int parallelism) { + return addNode(parent, child, parallelism, parent.getOutputStreams().iterator().next()); + } + + Node addNode(Node parent, Node child, String parentStreamId) { + return addNode(parent, child, parent.getParallelism(), parentStreamId); + } + + Node addNode(Node parent, Node child, int parallelism, String parentStreamId) { + graph.addVertex(child); + graph.addEdge(parent, child); + child.setParallelism(parallelism); + if (parent instanceof WindowNode || parent instanceof PartitionNode) { + child.addParentStream(parentNode(parent), parentStreamId); + } else { + child.addParentStream(parent, parentStreamId); + } + return child; + } + + private PriorityQueue queue() { + // min-heap + return new PriorityQueue<>(new Comparator() { + @Override + public int compare(Node n1, Node n2) { + return getPriority(n1.getClass()) - getPriority(n2.getClass()); + } + + private int getPriority(Class clazz) { + /* + * Nodes in the descending order of priority. + * ProcessorNode has the highest priority so that the topological order iterator + * will group as many processor nodes together as possible. + */ + Class[] p = new Class[]{ + ProcessorNode.class, + SpoutNode.class, + SinkNode.class, + PartitionNode.class, + WindowNode.class}; + for (int i = 0; i < p.length; i++) { + if (clazz.equals(p[i])) { + return i; + } + } + return Integer.MAX_VALUE; + } + }); + } + + private void handleProcessorNode(ProcessorNode processorNode, TopologyBuilder topologyBuilder) { + if (processorNode.getProcessor() instanceof StatefulProcessor) { + statefulProcessorCount++; + Set initialNodes = initialProcessors( + curGroup.isEmpty() ? Collections.singletonList(processorNode) : curGroup); + Set> windows = getWindowParams(initialNodes); + // if we get more than one stateful operation, we need to process the + // current group so that we have one stateful operation per stateful bolt + if (statefulProcessorCount > 1 || !windows.isEmpty()) { + if (!curGroup.isEmpty()) { + processCurGroup(topologyBuilder); + } else if (!windows.isEmpty()) { + // a stateful processor immediately follows a window specification + splitStatefulProcessor(processorNode, topologyBuilder); + } + statefulProcessorCount = 1; + } + } + curGroup.add(processorNode); + } + + /* + * force create a windowed bolt with identity nodes so that we don't + * have a stateful processor inside a windowed bolt. + */ + private void splitStatefulProcessor(ProcessorNode processorNode, TopologyBuilder topologyBuilder) { + for (Node parent : StreamUtil.getParents(graph, processorNode)) { + ProcessorNode identity = + new ProcessorNode(new MapProcessor<>(new IdentityFunction<>()), + UniqueIdGen.getInstance().getUniqueStreamId(), + parent.getOutputFields()); + addNode(parent, identity); + graph.removeEdge(parent, processorNode); + processorNode.removeParentStreams(parent); + addNode(identity, processorNode); + curGroup.add(identity); + } + processCurGroup(topologyBuilder); + } + + private void mayBeAddTsField() { + if (timestampFieldName != null) { + for (StreamBolt streamBolt : streamBolts.keySet()) { + streamBolt.setTimestampField(timestampFieldName); + } + } + } + + private void updateNodeGroupingInfo(PartitionNode partitionNode) { + if (partitionNode.getGroupingInfo() != null) { + for (Node parent : parentNodes(partitionNode)) { + for (String parentStream : partitionNode.getParentStreams(parent)) { + nodeGroupingInfo.put(parent, parentStream, partitionNode.getGroupingInfo()); + } + } + } + } + + private void updateWindowInfo(WindowNode windowNode) { + for (Node parent : parentNodes(windowNode)) { + windowInfo.put(parent, windowNode); + } + String tsField = windowNode.getWindowParams().getTimestampField(); + if (tsField != null) { + if (timestampFieldName != null && !tsField.equals(timestampFieldName)) { + throw new IllegalArgumentException("Cannot set different timestamp field names"); + } + timestampFieldName = tsField; + } + } + + private Node parentNode(Node curNode) { + Set parentNode = parentNodes(curNode); + if (parentNode.size() > 1) { + throw new IllegalArgumentException("Node " + curNode + " has more than one parent node."); + } + if (parentNode.isEmpty()) { + throw new IllegalArgumentException("Node " + curNode + " has no parent."); + } + return parentNode.iterator().next(); + } + + private Set parentNodes(Node curNode) { + Set nodes = new HashSet<>(); + for (Node parent : StreamUtil.getParents(graph, curNode)) { + if (parent instanceof ProcessorNode || parent instanceof SpoutNode) { + nodes.add(parent); + } else { + nodes.addAll(parentNodes(parent)); + } + } + return nodes; + } + + private void processCurGroup(TopologyBuilder topologyBuilder) { + if (curGroup.isEmpty()) { + return; + } + + String boltId = UniqueIdGen.getInstance().getUniqueBoltId(); + for (ProcessorNode processorNode : curGroup) { + processorNode.setComponentId(boltId); + processorNode.setWindowed(isWindowed(processorNode)); + processorNode.setWindowedParentStreams(getWindowedParentStreams(processorNode)); + } + final Set initialProcessors = initialProcessors(curGroup); + Set> windowParams = getWindowParams(initialProcessors); + if (windowParams.isEmpty()) { + if (hasStatefulProcessor(curGroup)) { + addStatefulBolt(topologyBuilder, boltId, initialProcessors); + } else { + addBolt(topologyBuilder, boltId, initialProcessors); + } + } else if (windowParams.size() == 1) { + addWindowedBolt(topologyBuilder, boltId, initialProcessors, windowParams.iterator().next()); + } else { + throw new IllegalStateException("More than one window config for current group " + curGroup); + } + curGroup.clear(); + } + + private boolean hasStatefulProcessor(List processorNodes) { + for (ProcessorNode node : processorNodes) { + if (node.getProcessor() instanceof StatefulProcessor) { + return true; + } + } + return false; + } + + private int getParallelism() { + Set parallelisms = new HashSet<>(Collections2.transform(curGroup, new Function() { + @Override + public Integer apply(ProcessorNode input) { + return input.getParallelism(); + } + })); + + if (parallelisms.size() > 1) { + throw new IllegalStateException("Current group does not have same parallelism " + curGroup); + } + + return parallelisms.isEmpty() ? 1 : parallelisms.iterator().next(); + } + + private Set> getWindowParams(Set initialProcessors) { + Set windowNodes = new HashSet<>(); + Set parents; + for (ProcessorNode processorNode : initialProcessors) { + if (processorNode.getProcessor() instanceof JoinProcessor) { + String leftStream = ((JoinProcessor) processorNode.getProcessor()).getLeftStream(); + parents = processorNode.getParents(leftStream); + } else { + parents = parentNodes(processorNode); + } + for (Node node : parents) { + if (windowInfo.containsKey(node)) { + windowNodes.add(windowInfo.get(node)); + } + } + } + + Set> windowParams = new HashSet<>(); + if (!windowNodes.isEmpty()) { + windowParams.addAll(new HashSet<>(Collections2.transform(windowNodes, new Function>() { + @Override + public Window apply(WindowNode input) { + return input.getWindowParams(); + } + }))); + } + return windowParams; + } + + private void addSpout(TopologyBuilder topologyBuilder, SpoutNode spout) { + topologyBuilder.setSpout(spout.getComponentId(), spout.getSpout(), spout.getParallelism()); + } + + private void addSink(TopologyBuilder topologyBuilder, SinkNode sinkNode) { + IComponent bolt = sinkNode.getBolt(); + BoltDeclarer boltDeclarer; + if (bolt instanceof IRichBolt) { + boltDeclarer = topologyBuilder.setBolt(sinkNode.getComponentId(), (IRichBolt) bolt, sinkNode.getParallelism()); + } else if (bolt instanceof IBasicBolt) { + boltDeclarer = topologyBuilder.setBolt(sinkNode.getComponentId(), (IBasicBolt) bolt, sinkNode.getParallelism()); + } else { + throw new IllegalArgumentException("Expect IRichBolt or IBasicBolt in addBolt"); + } + for (Node parent : parentNodes(sinkNode)) { + for (String stream : sinkNode.getParentStreams(parent)) { + declareStream(boltDeclarer, parent, stream, nodeGroupingInfo.get(parent, stream)); + } + } + } + + private StreamBolt addBolt(TopologyBuilder topologyBuilder, + String boltId, + Set initialProcessors) { + ProcessorBolt bolt = new ProcessorBolt(boltId, graph, curGroup); + BoltDeclarer boltDeclarer = topologyBuilder.setBolt(boltId, bolt, getParallelism()); + bolt.setStreamToInitialProcessors(wireBolt(curGroup, boltDeclarer, initialProcessors)); + streamBolts.put(bolt, boltDeclarer); + return bolt; + } + + private StreamBolt addStatefulBolt(TopologyBuilder topologyBuilder, + String boltId, + Set initialProcessors) { + StateQueryProcessor stateQueryProcessor = getStateQueryProcessor(); + StatefulProcessorBolt bolt; + if (stateQueryProcessor == null) { + bolt = new StatefulProcessorBolt<>(boltId, graph, curGroup); + BoltDeclarer boltDeclarer = topologyBuilder.setBolt(boltId, bolt, getParallelism()); + bolt.setStreamToInitialProcessors(wireBolt(curGroup, boltDeclarer, initialProcessors)); + streamBolts.put(bolt, boltDeclarer); + } else { + // state query is added to the existing stateful bolt + ProcessorNode updateStateNode = stateQueryProcessor.getStreamState().getUpdateStateNode(); + bolt = findStatefulProcessorBolt(updateStateNode); + for (ProcessorNode node : curGroup) { + node.setComponentId(bolt.getId()); + } + bolt.addNodes(curGroup); + bolt.addStreamToInitialProcessors(wireBolt(bolt.getNodes(), streamBolts.get(bolt), initialProcessors)); + } + return bolt; + } + + private StateQueryProcessor getStateQueryProcessor() { + for (ProcessorNode node : curGroup) { + if (node.getProcessor() instanceof StateQueryProcessor) { + return (StateQueryProcessor) node.getProcessor(); + } + } + return null; + } + + private StreamBolt addWindowedBolt(TopologyBuilder topologyBuilder, + String boltId, + Set initialProcessors, + Window windowParam) { + WindowedProcessorBolt bolt = new WindowedProcessorBolt(boltId, graph, curGroup, windowParam); + BoltDeclarer boltDeclarer = topologyBuilder.setBolt(boltId, bolt, getParallelism()); + bolt.setStreamToInitialProcessors(wireBolt(curGroup, boltDeclarer, initialProcessors)); + streamBolts.put(bolt, boltDeclarer); + return bolt; + } + + private StatefulProcessorBolt findStatefulProcessorBolt(ProcessorNode updateStateNode) { + for (StreamBolt bolt : streamBolts.keySet()) { + if (bolt instanceof StatefulProcessorBolt) { + StatefulProcessorBolt statefulProcessorBolt = (StatefulProcessorBolt) bolt; + if (statefulProcessorBolt.getNodes().contains(updateStateNode)) { + return statefulProcessorBolt; + } + } + } + throw new IllegalArgumentException("Could not find Stateful bolt for node " + updateStateNode); + } + + private Set getWindowedParentStreams(ProcessorNode processorNode) { + Set res = new HashSet<>(); + for (Node parent : parentNodes(processorNode)) { + if (parent instanceof ProcessorNode) { + ProcessorNode pn = (ProcessorNode) parent; + if (pn.isWindowed()) { + res.addAll(Collections2.filter(pn.getOutputStreams(), new Predicate() { + @Override + public boolean apply(String input) { + return !StreamUtil.isSinkStream(input); + } + })); + } + } + } + return res; + } + + private Multimap wireBolt(List curGroup, + BoltDeclarer boltDeclarer, + Set initialProcessors) { + LOG.debug("Wiring bolt with boltDeclarer {}, curGroup {}, initialProcessors {}, nodeGroupingInfo {}", + boltDeclarer, curGroup, initialProcessors, nodeGroupingInfo); + Multimap streamToInitialProcessor = ArrayListMultimap.create(); + Set curSet = new HashSet<>(curGroup); + for (ProcessorNode curNode : initialProcessors) { + for (Node parent : parentNodes(curNode)) { + if (curSet.contains(parent)) { + LOG.debug("Parent {} of curNode {} is in curGroup {}", parent, curNode, curGroup); + } else { + for (String stream : curNode.getParentStreams(parent)) { + declareStream(boltDeclarer, parent, stream, nodeGroupingInfo.get(parent, stream)); + // put global stream id for spouts + if (parent.getComponentId().startsWith("spout")) { + stream = parent.getComponentId() + stream; + } + streamToInitialProcessor.put(stream, curNode); + } + } + } + } + return streamToInitialProcessor; + } + + private void declareStream(BoltDeclarer boltDeclarer, Node parent, String streamId, GroupingInfo grouping) { + if (grouping == null) { + boltDeclarer.shuffleGrouping(parent.getComponentId(), streamId); + } else { + grouping.declareGrouping(boltDeclarer, parent.getComponentId(), streamId, grouping.getFields()); + } + } + + private Set initialProcessors(List curGroup) { + Set nodes = new HashSet<>(); + Set curSet = new HashSet<>(curGroup); + for (ProcessorNode node : curGroup) { + for (Node parent : parentNodes(node)) { + if (!(parent instanceof ProcessorNode) || !curSet.contains(parent)) { + nodes.add(node); + } + } + } + return nodes; + } + + private boolean isWindowed(Node curNode) { + for (Node parent : StreamUtil.getParents(graph, curNode)) { + if (parent instanceof WindowNode) { + return true; + } else if (parent instanceof ProcessorNode) { + ProcessorNode p = (ProcessorNode) parent; + if (p.isWindowed()) { + return true; + } + } else { + return (parent instanceof PartitionNode) && isWindowed(parent); + } + } + return false; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/StreamState.java b/storm-core/src/jvm/org/apache/storm/streams/StreamState.java new file mode 100644 index 00000000000..a4633f751d3 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/StreamState.java @@ -0,0 +1,43 @@ +/** + * 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.storm.streams; + +import java.io.Serializable; + +/** + * A wrapper for the stream state which can be used to + * query the state via {@link Stream#stateQuery(StreamState)} + * + * @param the key type + * @param the value type + */ +public class StreamState implements Serializable { + private final transient PairStream stream; + + StreamState(PairStream stream) { + this.stream = stream; + } + + public PairStream toPairStream() { + return stream; + } + + ProcessorNode getUpdateStateNode() { + return (ProcessorNode) stream.node; + } +} \ No newline at end of file diff --git a/storm-core/src/jvm/org/apache/storm/streams/StreamUtil.java b/storm-core/src/jvm/org/apache/storm/streams/StreamUtil.java new file mode 100644 index 00000000000..0531ff60bd3 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/StreamUtil.java @@ -0,0 +1,62 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.streams; + +import org.jgrapht.DirectedGraph; + +import java.util.ArrayList; +import java.util.List; + +public class StreamUtil { + @SuppressWarnings("unchecked") + public static List getParents(DirectedGraph graph, Node node) { + List incoming = new ArrayList<>(graph.incomingEdgesOf(node)); + List ret = new ArrayList<>(); + for (Edge e : incoming) { + ret.add((T) e.getSource()); + } + return ret; + } + + @SuppressWarnings("unchecked") + public static List getChildren(DirectedGraph graph, Node node) { + List outgoing = new ArrayList<>(graph.outgoingEdgesOf(node)); + List ret = new ArrayList<>(); + for (Edge e : outgoing) { + ret.add((T) e.getTarget()); + } + return ret; + } + + + public static boolean isSinkStream(String streamId) { + return streamId.endsWith("__sink"); + } + + public static String getSinkStream(String streamId) { + return streamId + "__sink"; + } + + public static boolean isPunctuation(Object value) { + if (value instanceof Pair) { + value = ((Pair) value).getFirst(); + } + return WindowNode.PUNCTUATION.equals(value); + } + +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/StreamsEdgeFactory.java b/storm-core/src/jvm/org/apache/storm/streams/StreamsEdgeFactory.java new file mode 100644 index 00000000000..0078690b167 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/StreamsEdgeFactory.java @@ -0,0 +1,29 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.streams; + +import org.jgrapht.EdgeFactory; + +import java.io.Serializable; + +class StreamsEdgeFactory implements EdgeFactory, Serializable { + @Override + public Edge createEdge(Node sourceVertex, Node targetVertex) { + return new Edge(sourceVertex, targetVertex); + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/Tuple3.java b/storm-core/src/jvm/org/apache/storm/streams/Tuple3.java new file mode 100644 index 00000000000..77973f2746b --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/Tuple3.java @@ -0,0 +1,49 @@ +/** + * 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.storm.streams; + +/** + * A tuple of three elements along the lines of Scala's Tuple. + * + * @param the type of the first element + * @param the type of the second element + * @param the type of the third element + */ +public class Tuple3 { + public final T1 _1; + public final T2 _2; + public final T3 _3; + + /** + * Constructs a new tuple of three elements. + * + * @param _1 the first element + * @param _2 the second element + * @param _3 the third element + */ + public Tuple3(T1 _1, T2 _2, T3 _3) { + this._1 = _1; + this._2 = _2; + this._3 = _3; + } + + @Override + public String toString() { + return "(" + _1 + "," + _2 + "," + _3 + ")"; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/UniqueIdGen.java b/storm-core/src/jvm/org/apache/storm/streams/UniqueIdGen.java new file mode 100644 index 00000000000..3cbd141d122 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/UniqueIdGen.java @@ -0,0 +1,56 @@ +/** + * 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.storm.streams; + +class UniqueIdGen { + private int streamCounter = 0; + private int spoutCounter = 0; + private int boltCounter = 0; + private static final UniqueIdGen instance = new UniqueIdGen(); + + private UniqueIdGen() { + } + + static UniqueIdGen getInstance() { + return instance; + } + + String getUniqueStreamId() { + streamCounter++; + return "s" + streamCounter; + } + + String getUniqueBoltId() { + boltCounter++; + return "bolt" + boltCounter; + } + + String getUniqueSpoutId() { + spoutCounter++; + return "spout" + spoutCounter; + } + + // for unit tests + void reset() { + streamCounter = 0; + spoutCounter = 0; + boltCounter = 0; + } + +} + diff --git a/storm-core/src/jvm/org/apache/storm/streams/WindowNode.java b/storm-core/src/jvm/org/apache/storm/streams/WindowNode.java new file mode 100644 index 00000000000..a0e831a8492 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/WindowNode.java @@ -0,0 +1,38 @@ +/** + * 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.storm.streams; + +import org.apache.storm.streams.windowing.Window; +import org.apache.storm.tuple.Fields; + +/** + * Node that captures the windowing configurations. + */ +public class WindowNode extends Node { + private final Window windowParams; + public static final String PUNCTUATION = "__punctuation"; + + WindowNode(Window windowParams, String outputStream, Fields outputFields) { + super(outputStream, outputFields); + this.windowParams = windowParams; + } + + Window getWindowParams() { + return windowParams; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/WindowedProcessorBolt.java b/storm-core/src/jvm/org/apache/storm/streams/WindowedProcessorBolt.java new file mode 100644 index 00000000000..397134699e8 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/WindowedProcessorBolt.java @@ -0,0 +1,137 @@ +/** + * 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.storm.streams; + +import com.google.common.collect.Multimap; +import org.apache.storm.streams.windowing.SlidingWindows; +import org.apache.storm.streams.windowing.TumblingWindows; +import org.apache.storm.streams.windowing.Window; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.topology.base.BaseWindowedBolt; +import org.apache.storm.tuple.Tuple; +import org.apache.storm.windowing.TupleWindow; +import org.jgrapht.DirectedGraph; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Date; +import java.util.List; +import java.util.Map; + +import static org.apache.storm.streams.WindowNode.PUNCTUATION; + +/** + * Stream bolt that executes windowing operations. + */ +class WindowedProcessorBolt extends BaseWindowedBolt implements StreamBolt { + private static final Logger LOG = LoggerFactory.getLogger(WindowedProcessorBolt.class); + private final ProcessorBoltDelegate delegate; + private final Window window; + + WindowedProcessorBolt(String id, DirectedGraph graph, + List nodes, + Window window) { + delegate = new ProcessorBoltDelegate(id, graph, nodes); + this.window = window; + setWindowConfig(); + } + + @Override + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + delegate.prepare(stormConf, context, collector); + } + + @Override + public void execute(TupleWindow inputWindow) { + LOG.trace("Window triggered at {}, inputWindow {}", new Date(), inputWindow); + if (delegate.isEventTimestamp()) { + delegate.setEventTimestamp(inputWindow.getTimestamp()); + } + for (Tuple tuple : inputWindow.get()) { + Pair valueAndStream = delegate.getValueAndStream(tuple); + if (!StreamUtil.isPunctuation(valueAndStream.getFirst())) { + delegate.process(valueAndStream.getFirst(), valueAndStream.getSecond()); + } + } + for (String stream : delegate.getInitialStreams()) { + delegate.process(PUNCTUATION, stream); + } + } + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + delegate.declareOutputFields(declarer); + } + + @Override + public void setTimestampField(String fieldName) { + delegate.setTimestampField(fieldName); + } + + @Override + public String getId() { + return delegate.getId(); + } + + private void setWindowConfig() { + if (window instanceof SlidingWindows) { + setSlidingWindowParams(window.getWindowLength(), window.getSlidingInterval()); + } else if (window instanceof TumblingWindows) { + setTumblingWindowParams(window.getWindowLength()); + } + if (window.getTimestampField() != null) { + withTimestampField(window.getTimestampField()); + } + if (window.getLag() != null) { + withLag(window.getLag()); + } + if (window.getLateTupleStream() != null) { + withLateTupleStream(window.getLateTupleStream()); + } + } + + private void setSlidingWindowParams(Object windowLength, Object slidingInterval) { + if (windowLength instanceof Count) { + if (slidingInterval instanceof Count) { + withWindow((Count) windowLength, (Count) slidingInterval); + } else if (slidingInterval instanceof Duration) { + withWindow((Count) windowLength, (Duration) slidingInterval); + } + } else if (windowLength instanceof Duration) { + if (slidingInterval instanceof Count) { + withWindow((Duration) windowLength, (Count) slidingInterval); + } else if (slidingInterval instanceof Duration) { + withWindow((Duration) windowLength, (Duration) slidingInterval); + } + } + } + + private void setTumblingWindowParams(Object windowLength) { + if (windowLength instanceof Count) { + withTumblingWindow((Count) windowLength); + } else if (windowLength instanceof Duration) { + withTumblingWindow((Duration) windowLength); + } + } + + void setStreamToInitialProcessors(Multimap streamToInitialProcessors) { + delegate.setStreamToInitialProcessors(streamToInitialProcessors); + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/Aggregator.java b/storm-core/src/jvm/org/apache/storm/streams/operations/Aggregator.java new file mode 100644 index 00000000000..e3feaf4e3a0 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/operations/Aggregator.java @@ -0,0 +1,42 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.streams.operations; + +/** + * Interface for aggregating values. + * + * @param the original value type + * @param the aggregated value type + */ +public interface Aggregator extends Operation { + /** + * The initial value of the aggregate to start with. + * + * @return the initial value + */ + R init(); + + /** + * Returns a new aggregate by applying the value with the current aggregate. + * + * @param value the value to aggregate + * @param aggregate the current aggregate + * @return the new aggregate + */ + R apply(T value, R aggregate); +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/Consumer.java b/storm-core/src/jvm/org/apache/storm/streams/operations/Consumer.java new file mode 100644 index 00000000000..84653abb2fb --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/operations/Consumer.java @@ -0,0 +1,32 @@ +/** + * 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.storm.streams.operations; + +/** + * Represents an operation that accepts a single input argument and returns no result. + * + * @param the type of the input argument + */ +public interface Consumer extends Operation { + /** + * Performs an operation on the given argument. + * + * @param input the input + */ + void accept(T input); +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/FlatMapFunction.java b/storm-core/src/jvm/org/apache/storm/streams/operations/FlatMapFunction.java new file mode 100644 index 00000000000..bcacd082f85 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/operations/FlatMapFunction.java @@ -0,0 +1,27 @@ +/** + * 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.storm.streams.operations; + +/** + * A function that accepts one argument and returns an {@link Iterable} of elements as its result. + * + * @param the type of the input to the function + * @param the result type. An iterable of this type is returned from this function + */ +public interface FlatMapFunction extends Function> { +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/Function.java b/storm-core/src/jvm/org/apache/storm/streams/operations/Function.java new file mode 100644 index 00000000000..7cef0a6b2e6 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/operations/Function.java @@ -0,0 +1,34 @@ +/** + * 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.storm.streams.operations; + +/** + * Represents a function that accepts one argument and produces a result. + * + * @param the type of the input to the function + * @param the type of the result of the function + */ +public interface Function extends Operation { + /** + * Applies this function to the given argument. + * + * @param input the input to the function + * @return the function result + */ + R apply(T input); +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/IdentityFunction.java b/storm-core/src/jvm/org/apache/storm/streams/operations/IdentityFunction.java new file mode 100644 index 00000000000..abb9327bef6 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/operations/IdentityFunction.java @@ -0,0 +1,31 @@ +/** + * 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.storm.streams.operations; + +/** + * A {@link Function} that returns the input argument itself as the result. + * + * @param the type of the input + */ +public class IdentityFunction implements Function { + + @Override + public T apply(T input) { + return input; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/Operation.java b/storm-core/src/jvm/org/apache/storm/streams/operations/Operation.java new file mode 100644 index 00000000000..77dbe1ed4f4 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/operations/Operation.java @@ -0,0 +1,26 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.streams.operations; + +import java.io.Serializable; + +/** + * The parent interface for any kind of streaming operation. + */ +public interface Operation extends Serializable { +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/PairFlatMapFunction.java b/storm-core/src/jvm/org/apache/storm/streams/operations/PairFlatMapFunction.java new file mode 100644 index 00000000000..376c1ba1176 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/operations/PairFlatMapFunction.java @@ -0,0 +1,30 @@ +/** + * 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.storm.streams.operations; + +import org.apache.storm.streams.Pair; + +/** + * A function that accepts one argument and returns an {@link Iterable} of {@link Pair} as its result. + * + * @param the type of the input to the function + * @param the key type of the key-value pairs produced as a result + * @param the value type of the key-value pairs produced as a result + */ +public interface PairFlatMapFunction extends FlatMapFunction> { +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/PairFunction.java b/storm-core/src/jvm/org/apache/storm/streams/operations/PairFunction.java new file mode 100644 index 00000000000..b153ff18b21 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/operations/PairFunction.java @@ -0,0 +1,30 @@ +/** + * 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.storm.streams.operations; + +import org.apache.storm.streams.Pair; + +/** + * A function that accepts an argument and produces a key-value Pair. + * + * @param the type of the input to the function + * @param the key type + * @param the value type + */ +public interface PairFunction extends Function> { +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/PairValueJoiner.java b/storm-core/src/jvm/org/apache/storm/streams/operations/PairValueJoiner.java new file mode 100644 index 00000000000..ca811018c20 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/operations/PairValueJoiner.java @@ -0,0 +1,40 @@ +/** + * 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.storm.streams.operations; + +import org.apache.storm.streams.Pair; + +/** + * A {@link ValueJoiner} that joins two values to produce a {@link Pair} of the two values as the result. + * + * @param the type of the first value + * @param the type of the second value + */ +public class PairValueJoiner implements ValueJoiner> { + /** + * Joins two values and produces a {@link Pair} of the values as the result. + * + * @param value1 the first value + * @param value2 the second value + * @return a pair of the first and second value + */ + @Override + public Pair apply(V1 value1, V2 value2) { + return Pair.of(value1, value2); + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/Predicate.java b/storm-core/src/jvm/org/apache/storm/streams/operations/Predicate.java new file mode 100644 index 00000000000..ae1be2de727 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/operations/Predicate.java @@ -0,0 +1,33 @@ +/** + * 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.storm.streams.operations; + +/** + * Represents a predicate (boolean-valued function) of a value. + * + * @param the value type + */ +public interface Predicate extends Operation { + /** + * Evaluates this predicate on the given argument. + * + * @param input the input argument + * @return true if the input matches the predicate, false otherwise + */ + boolean test(T input); +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/PrintConsumer.java b/storm-core/src/jvm/org/apache/storm/streams/operations/PrintConsumer.java new file mode 100644 index 00000000000..d301dbfad93 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/operations/PrintConsumer.java @@ -0,0 +1,30 @@ +/** + * 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.storm.streams.operations; + +/** + * A consumer that prints the input to stdout. + * + * @param the value type + */ +public class PrintConsumer implements Consumer { + @Override + public void accept(T input) { + System.out.println(input); + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/Reducer.java b/storm-core/src/jvm/org/apache/storm/streams/operations/Reducer.java new file mode 100644 index 00000000000..04ee70d07c7 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/operations/Reducer.java @@ -0,0 +1,35 @@ +/** + * 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.storm.streams.operations; + +/** + * The {@link Reducer} performs an operation on two values of the same type producing a result of the same + * type. + * + * @param the type of the arguments and the result + */ +public interface Reducer extends Operation { + /** + * Applies this function to the given arguments. + * + * @param arg1 the first argument + * @param arg2 the second argument + * @return the result + */ + T apply(T arg1, T arg2); +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/ValueJoiner.java b/storm-core/src/jvm/org/apache/storm/streams/operations/ValueJoiner.java new file mode 100644 index 00000000000..003d097976a --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/operations/ValueJoiner.java @@ -0,0 +1,36 @@ +/** + * 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.storm.streams.operations; + +/** + * An interface for joining two values to produce a result. + * + * @param the type of the first value + * @param the type of the second value + * @param the result type + */ +public interface ValueJoiner extends Operation { + /** + * Joins two values and produces a result. + * + * @param value1 the first value + * @param value2 the second value + * @return the result + */ + R apply(V1 value1, V2 value2); +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Count.java b/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Count.java new file mode 100644 index 00000000000..fd02d2a0247 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Count.java @@ -0,0 +1,37 @@ +/** + * 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.storm.streams.operations.aggregators; + +import org.apache.storm.streams.operations.Aggregator; + +/** + * Computes the count of values. + * + * @param the value type + */ +public class Count implements Aggregator { + @Override + public Long init() { + return 0L; + } + + @Override + public Long apply(T value, Long aggregate) { + return aggregate + 1; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Sum.java b/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Sum.java new file mode 100644 index 00000000000..e23207562d8 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Sum.java @@ -0,0 +1,35 @@ +/** + * 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.storm.streams.operations.aggregators; + +import org.apache.storm.streams.operations.Aggregator; + +/** + * Computes the long sum of the input values + */ +public class Sum implements Aggregator { + @Override + public Long init() { + return 0L; + } + + @Override + public Long apply(Number value, Long aggregate) { + return value.longValue() + aggregate; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/PairValueMapper.java b/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/PairValueMapper.java new file mode 100644 index 00000000000..1490e514df7 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/PairValueMapper.java @@ -0,0 +1,51 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.streams.operations.mappers; + +import org.apache.storm.streams.Pair; +import org.apache.storm.streams.operations.PairFunction; +import org.apache.storm.tuple.Tuple; + +/** + * Extracts a typed key-value pair from a tuple. + * + * @param the key type + * @param the value type + */ +public class PairValueMapper implements TupleValueMapper>, PairFunction { + private final int keyIndex; + private final int valueIndex; + + /** + * Constructs a new {@link PairValueMapper} that constructs a pair + * from a tuple based on the key and value index. + * + * @param keyIndex the key index + * @param valueIndex the value index + */ + public PairValueMapper(int keyIndex, int valueIndex) { + this.keyIndex = keyIndex; + this.valueIndex = valueIndex; + } + + @SuppressWarnings("unchecked") + @Override + public Pair apply(Tuple input) { + return Pair.of((K) input.getValue(keyIndex), (V) input.getValue(valueIndex)); + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/TupleValueMapper.java b/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/TupleValueMapper.java new file mode 100644 index 00000000000..6ffbd886d28 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/TupleValueMapper.java @@ -0,0 +1,30 @@ +/** + * 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.storm.streams.operations.mappers; + +import org.apache.storm.streams.operations.Function; +import org.apache.storm.tuple.Tuple; + +/** + * A generic interface for mapping a {@link Tuple} to typed + * values. + * + * @param the result type + */ +public interface TupleValueMapper extends Function { +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/ValueMapper.java b/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/ValueMapper.java new file mode 100644 index 00000000000..0468d0ddce5 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/ValueMapper.java @@ -0,0 +1,45 @@ +/** + * 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.storm.streams.operations.mappers; + +import org.apache.storm.tuple.Tuple; + +/** + * Extracts a single typed value from a tuple. + * + * @param the value type + */ +public class ValueMapper implements TupleValueMapper { + private final int index; + + /** + * Constructs a new {@link ValueMapper} that extracts + * the value at index 'i' from a tuple. + * + * @param index the index + */ + public ValueMapper(int index) { + this.index = index; + } + + @SuppressWarnings("unchecked") + @Override + public T apply(Tuple input) { + return (T) input.getValue(index); + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/ValuesMapper.java b/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/ValuesMapper.java new file mode 100644 index 00000000000..f1d8e3b1b00 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/ValuesMapper.java @@ -0,0 +1,48 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.streams.operations.mappers; + +import org.apache.storm.tuple.Tuple; +import org.apache.storm.tuple.Values; + +/** + * Constructs a {@link Values} from a {@link Tuple} + * based on indicies. + */ +public class ValuesMapper implements TupleValueMapper { + private final int[] indices; + + /** + * Constructs a new {@link ValuesMapper} that extracts + * value from a {@link Tuple} at specified indices + * + * @param indices the indices + */ + public ValuesMapper(int... indices) { + this.indices = indices; + } + + @Override + public Values apply(Tuple input) { + Values values = new Values(); + for (int i : indices) { + values.add(input.getValue(i)); + } + return values; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateByKeyProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateByKeyProcessor.java new file mode 100644 index 00000000000..d53a52d4f06 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateByKeyProcessor.java @@ -0,0 +1,54 @@ +/** + * 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.storm.streams.processors; + +import org.apache.storm.streams.Pair; +import org.apache.storm.streams.operations.Aggregator; + +import java.util.HashMap; +import java.util.Map; + +public class AggregateByKeyProcessor extends BaseProcessor> implements BatchProcessor { + private final Aggregator aggregator; + private final Map state = new HashMap<>(); + + public AggregateByKeyProcessor(Aggregator aggregator) { + this.aggregator = aggregator; + } + + @Override + public void execute(Pair input) { + K key = input.getFirst(); + V val = input.getSecond(); + R agg = state.get(key); + if (agg == null) { + agg = aggregator.init(); + } + state.put(key, aggregator.apply(val, agg)); + mayBeForwardAggUpdate(Pair.of(key, state.get(key))); + } + + @Override + public void finish() { + for (Map.Entry entry : state.entrySet()) { + context.forward(Pair.of(entry.getKey(), entry.getValue())); + } + state.clear(); + } + +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateProcessor.java new file mode 100644 index 00000000000..c5a1906202a --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateProcessor.java @@ -0,0 +1,45 @@ +/** + * 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.storm.streams.processors; + +import org.apache.storm.streams.operations.Aggregator; + +public class AggregateProcessor extends BaseProcessor implements BatchProcessor { + private final Aggregator aggregator; + private R state; + + public AggregateProcessor(Aggregator aggregator) { + this.aggregator = aggregator; + } + + @Override + public void execute(T input) { + if (state == null) { + state = aggregator.init(); + } + R curAggregate = (state != null) ? state : aggregator.init(); + state = aggregator.apply(input, curAggregate); + mayBeForwardAggUpdate(state); + } + + @Override + public void finish() { + context.forward(state); + state = null; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/BaseProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/BaseProcessor.java new file mode 100644 index 00000000000..3ea469c0bab --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/BaseProcessor.java @@ -0,0 +1,107 @@ +/** + * 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.storm.streams.processors; + +import java.util.HashSet; +import java.util.Set; + +import static org.apache.storm.streams.WindowNode.PUNCTUATION; + +/** + * Base implementation of the {@link Processor} interface that provides + * convenience methods {@link #execute(Object)} and {@link #finish()}. + * + * @param + */ +abstract class BaseProcessor implements Processor { + private final Set punctuationState = new HashSet<>(); + protected ProcessorContext context; + + /** + * {@inheritDoc} + */ + @Override + public void init(ProcessorContext context) { + this.context = context; + } + + /** + * {@inheritDoc} + * Processors that do not care about the source stream should + * override {@link BaseProcessor#execute(Object)} instead. + */ + @Override + public void execute(T input, String streamId) { + execute(input); + } + + /** + * {@inheritDoc} + */ + @Override + public void punctuate(String stream) { + if ((stream == null) || shouldPunctuate(stream)) { + finish(); + context.forward(PUNCTUATION); + punctuationState.clear(); + } + } + + /** + * Execute some operation on the input value. Sub classes can override + * this when then don't care about the source stream from where the + * input is received. + * + * @param input the input + */ + protected void execute(T input) { + // NOOP + } + + /** + * This is triggered to signal the end of the current + * batch of values. Sub classes can override this to + * emit the result of a batch of values, for e.g. to emit + * the result of an aggregate or join operation on a batch + * of values. If a processor does per-value operation like + * filter, map etc, they can choose to ignore this. + */ + protected void finish() { + // NOOP + } + + /** + * Forwards the result update to downstream processors. Processors + * that operate on a batch of tuples, like aggregation, join etc + * can use this to emit the partial results on each input + * if they are operating in non-windowed mode. + * + * @param result the result + * @param the result type + */ + protected final void mayBeForwardAggUpdate(R result) { + if (!context.isWindowed()) { + context.forward(result); + } + } + + private boolean shouldPunctuate(String parentStream) { + punctuationState.add(parentStream); + return punctuationState.equals(context.getWindowedParentStreams()); + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/BatchProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/BatchProcessor.java new file mode 100644 index 00000000000..f9a39b42e26 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/BatchProcessor.java @@ -0,0 +1,25 @@ +/** + * 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.storm.streams.processors; + +/** + * Top level marker interface for processors that computes + * results for a batch of tuples like Aggregate, Join etc. + */ +public interface BatchProcessor { +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/BranchProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/BranchProcessor.java new file mode 100644 index 00000000000..f8bc739d4a9 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/BranchProcessor.java @@ -0,0 +1,41 @@ +/** + * 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.storm.streams.processors; + +import org.apache.storm.streams.operations.Predicate; + +import java.util.HashMap; +import java.util.Map; + +public class BranchProcessor extends BaseProcessor { + private final Map, String> predicateToStream = new HashMap<>(); + + public void addPredicate(Predicate predicate, String stream) { + predicateToStream.put(predicate, stream); + } + + @Override + public void execute(T input) { + for (Map.Entry, String> entry : predicateToStream.entrySet()) { + if (entry.getKey().test(input)) { + context.forward(input, entry.getValue()); + break; + } + } + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/ChainedProcessorContext.java b/storm-core/src/jvm/org/apache/storm/streams/processors/ChainedProcessorContext.java new file mode 100644 index 00000000000..c92d9928bce --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/ChainedProcessorContext.java @@ -0,0 +1,66 @@ +/** + * 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.storm.streams.processors; + +import org.apache.storm.streams.ProcessorNode; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Set; + +/** + * A composite context that holds a chain of {@link ProcessorContext} + */ +public class ChainedProcessorContext implements ProcessorContext { + private final ProcessorNode processorNode; + private final List contexts; + + public ChainedProcessorContext(ProcessorNode processorNode, List contexts) { + this.processorNode = processorNode; + this.contexts = new ArrayList<>(contexts); + } + + public ChainedProcessorContext(ProcessorNode processorNode, ProcessorContext... contexts) { + this(processorNode, Arrays.asList(contexts)); + } + + @Override + public void forward(T input) { + for (ProcessorContext context : contexts) { + context.forward(input); + } + } + + @Override + public void forward(T input, String stream) { + for (ProcessorContext context : contexts) { + context.forward(input, stream); + } + } + + @Override + public boolean isWindowed() { + return processorNode.isWindowed(); + } + + @Override + public Set getWindowedParentStreams() { + return processorNode.getWindowedParentStreams(); + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/EmittingProcessorContext.java b/storm-core/src/jvm/org/apache/storm/streams/processors/EmittingProcessorContext.java new file mode 100644 index 00000000000..d841080db3b --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/EmittingProcessorContext.java @@ -0,0 +1,170 @@ +/** + * 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.storm.streams.processors; + +import com.google.common.base.Function; +import com.google.common.collect.Collections2; +import org.apache.storm.streams.Pair; +import org.apache.storm.streams.ProcessorNode; +import org.apache.storm.streams.RefCountedTuple; +import org.apache.storm.streams.StreamUtil; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Tuple; +import org.apache.storm.tuple.Values; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.apache.storm.streams.WindowNode.PUNCTUATION; + +/** + * A context that emits the results to downstream processors which are in + * another bolt. + */ +public class EmittingProcessorContext implements ProcessorContext { + private static final Logger LOG = LoggerFactory.getLogger(EmittingProcessorContext.class); + private final ProcessorNode processorNode; + private final String outputStreamId; + private final OutputCollector collector; + private final Fields outputFields; + private final Values punctuation; + private final List anchors = new ArrayList<>(); + private boolean emitPunctuation = true; + private long eventTimestamp; + private String timestampField; + + public EmittingProcessorContext(ProcessorNode processorNode, OutputCollector collector, String outputStreamId) { + this.processorNode = processorNode; + this.outputStreamId = outputStreamId; + this.collector = collector; + outputFields = processorNode.getOutputFields(); + punctuation = createPunctuation(); + } + + @Override + public void forward(T input) { + if (input instanceof Pair) { + Pair value = (Pair) input; + emit(new Values(value.getFirst(), value.getSecond())); + } else if (PUNCTUATION.equals(input)) { + if (emitPunctuation) { + emit(punctuation); + } else { + LOG.debug("Not emitting punctuation since emitPunctuation is false"); + } + maybeAck(); + } else { + emit(new Values(input)); + } + } + + @Override + public void forward(T input, String stream) { + if (stream.equals(outputStreamId)) { + forward(input); + } + } + + @Override + public boolean isWindowed() { + return processorNode.isWindowed(); + } + + @Override + public Set getWindowedParentStreams() { + return processorNode.getWindowedParentStreams(); + } + + public void setEmitPunctuation(boolean emitPunctuation) { + this.emitPunctuation = emitPunctuation; + } + + public void setTimestampField(String fieldName) { + timestampField = fieldName; + } + + public void setAnchor(RefCountedTuple anchor) { + if (processorNode.isWindowed() && processorNode.isBatch()) { + anchor.increment(); + anchors.add(anchor); + } else { + if (anchors.isEmpty()) { + anchors.add(anchor); + } else { + anchors.set(0, anchor); + } + /* + * track punctuation in non-batch mode so that the + * punctuation is acked after all the processors have emitted the punctuation downstream. + */ + if (StreamUtil.isPunctuation(anchor.tuple().getValue(0))) { + anchor.increment(); + } + } + } + + public void setEventTimestamp(long timestamp) { + this.eventTimestamp = timestamp; + } + + private Values createPunctuation() { + Values values = new Values(); + for (int i = 0; i < outputFields.size(); i++) { + values.add(PUNCTUATION); + } + return values; + } + + private void maybeAck() { + if (!anchors.isEmpty()) { + for (RefCountedTuple anchor : anchors) { + anchor.decrement(); + if (anchor.shouldAck()) { + LOG.debug("Acking {} ", anchor); + collector.ack(anchor.tuple()); + anchor.setAcked(); + } + } + anchors.clear(); + } + } + + private Collection tuples(Collection anchors) { + return anchors.stream().map(RefCountedTuple::tuple).collect(Collectors.toList()); + } + + private void emit(Values values) { + if (timestampField != null) { + values.add(eventTimestamp); + } + if (anchors.isEmpty()) { + // for windowed bolt, windowed output collector will do the anchoring/acking + LOG.debug("Emit un-anchored, outputStreamId: {}, values: {}", outputStreamId, values); + collector.emit(outputStreamId, values); + } else { + LOG.debug("Emit, outputStreamId: {}, anchors: {}, values: {}", outputStreamId, anchors, values); + collector.emit(outputStreamId, tuples(anchors), values); + } + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/FilterProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/FilterProcessor.java new file mode 100644 index 00000000000..f245a445b46 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/FilterProcessor.java @@ -0,0 +1,35 @@ +/** + * 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.storm.streams.processors; + +import org.apache.storm.streams.operations.Predicate; + +public class FilterProcessor extends BaseProcessor { + private final Predicate predicate; + + public FilterProcessor(Predicate predicate) { + this.predicate = predicate; + } + + @Override + public void execute(T input) { + if (predicate.test(input)) { + context.forward(input); + } + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/FlatMapProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/FlatMapProcessor.java new file mode 100644 index 00000000000..fdc86fee130 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/FlatMapProcessor.java @@ -0,0 +1,35 @@ +/** + * 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.storm.streams.processors; + +import org.apache.storm.streams.operations.FlatMapFunction; + +public class FlatMapProcessor extends BaseProcessor { + private final FlatMapFunction function; + + public FlatMapProcessor(FlatMapFunction function) { + this.function = function; + } + + @Override + public void execute(T input) { + for (R res : function.apply(input)) { + context.forward(res); + } + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/FlatMapValuesProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/FlatMapValuesProcessor.java new file mode 100644 index 00000000000..92953a34602 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/FlatMapValuesProcessor.java @@ -0,0 +1,36 @@ +/** + * 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.storm.streams.processors; + +import org.apache.storm.streams.Pair; +import org.apache.storm.streams.operations.FlatMapFunction; + +public class FlatMapValuesProcessor extends BaseProcessor> { + private final FlatMapFunction function; + + public FlatMapValuesProcessor(FlatMapFunction function) { + this.function = function; + } + + @Override + protected void execute(Pair input) { + for (R res : function.apply(input.getSecond())) { + context.forward(Pair.of(input.getFirst(), res)); + } + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/ForEachProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/ForEachProcessor.java new file mode 100644 index 00000000000..323b53288c4 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/ForEachProcessor.java @@ -0,0 +1,33 @@ +/** + * 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.storm.streams.processors; + +import org.apache.storm.streams.operations.Consumer; + +public class ForEachProcessor extends BaseProcessor { + private final Consumer consumer; + + public ForEachProcessor(Consumer consumer) { + this.consumer = consumer; + } + + @Override + public void execute(T input) { + consumer.accept(input); + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/ForwardingProcessorContext.java b/storm-core/src/jvm/org/apache/storm/streams/processors/ForwardingProcessorContext.java new file mode 100644 index 00000000000..49eb836391f --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/ForwardingProcessorContext.java @@ -0,0 +1,102 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.streams.processors; + +import com.google.common.collect.Multimap; +import org.apache.storm.streams.ProcessorNode; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Set; + +import static org.apache.storm.streams.WindowNode.PUNCTUATION; + +/** + * A context that emits the results to downstream processors which are in + * the same bolt. + */ +public class ForwardingProcessorContext implements ProcessorContext { + private static final Logger LOG = LoggerFactory.getLogger(ForwardingProcessorContext.class); + private final ProcessorNode processorNode; + private final Multimap streamToChildren; + private final Set streams; + + public ForwardingProcessorContext(ProcessorNode processorNode, Multimap streamToChildren) { + this.processorNode = processorNode; + this.streamToChildren = streamToChildren; + this.streams = streamToChildren.keySet(); + } + + @Override + public void forward(T input) { + if (PUNCTUATION.equals(input)) { + finishAllStreams(); + } else { + executeAllStreams(input); + } + } + + @Override + public void forward(T input, String stream) { + if (PUNCTUATION.equals(input)) { + finish(stream); + } else { + execute(input, stream); + } + } + + @Override + public boolean isWindowed() { + return processorNode.isWindowed(); + } + + @Override + public Set getWindowedParentStreams() { + return processorNode.getWindowedParentStreams(); + } + + private void finishAllStreams() { + for (String stream : streams) { + finish(stream); + } + } + + @SuppressWarnings("unchecked") + private void finish(String stream) { + for (ProcessorNode node : streamToChildren.get(stream)) { + LOG.debug("Punctuating processor: {}", node); + Processor processor = (Processor) node.getProcessor(); + processor.punctuate(stream); + } + } + + private void executeAllStreams(T input) { + for (String stream : streams) { + execute(input, stream); + } + } + + @SuppressWarnings("unchecked") + private void execute(T input, String stream) { + for (ProcessorNode node : streamToChildren.get(stream)) { + LOG.debug("Forward input: {} to processor node: {}", input, node); + Processor processor = (Processor) node.getProcessor(); + processor.execute(input, stream); + } + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/JoinProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/JoinProcessor.java new file mode 100644 index 00000000000..d56cfea9ad1 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/JoinProcessor.java @@ -0,0 +1,112 @@ +/** + * 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.storm.streams.processors; + +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.Multimap; +import org.apache.storm.streams.Pair; +import org.apache.storm.streams.Tuple3; +import org.apache.storm.streams.operations.ValueJoiner; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** + * Provides equi-join implementation based on simple hash-join. + */ +public class JoinProcessor extends BaseProcessor> implements BatchProcessor { + private final ValueJoiner valueJoiner; + private final String leftStream; + private final String rightStream; + private final List> leftRows = new ArrayList<>(); + private final List> rightRows = new ArrayList<>(); + + public JoinProcessor(String leftStream, String rightStream, ValueJoiner valueJoiner) { + this.valueJoiner = valueJoiner; + this.leftStream = leftStream; + this.rightStream = rightStream; + } + + @Override + public void execute(Pair input, String sourceStream) { + K key = input.getFirst(); + if (sourceStream.equals(leftStream)) { + V1 val = (V1) input.getSecond(); + Pair pair = Pair.of(key, val); + leftRows.add(pair); + if (!context.isWindowed()) { + joinAndForward(Collections.singletonList(pair), rightRows); + } + } else if (sourceStream.equals(rightStream)) { + V2 val = (V2) input.getSecond(); + Pair pair = Pair.of(key, val); + rightRows.add(pair); + if (!context.isWindowed()) { + joinAndForward(leftRows, Collections.singletonList(pair)); + } + } + } + + @Override + public void finish() { + joinAndForward(leftRows, rightRows); + leftRows.clear(); + rightRows.clear(); + } + + public String getLeftStream() { + return leftStream; + } + + public String getRightStream() { + return rightStream; + } + + private void joinAndForward(List> leftRows, List> rightRows) { + if (leftRows.size() <= rightRows.size()) { + for (Tuple3 res : join(getJoinTable(leftRows), rightRows)) { + context.forward(Pair.of(res._1, valueJoiner.apply(res._2, res._3))); + } + } else { + for (Tuple3 res : join(getJoinTable(rightRows), leftRows)) { + context.forward(Pair.of(res._1, valueJoiner.apply(res._3, res._2))); + } + } + } + + private List> join(Multimap tab, List> rows) { + List> res = new ArrayList<>(); + for (Pair row : rows) { + for (T1 mapValue : tab.get(row.getFirst())) { + if (mapValue != null) { + res.add(new Tuple3<>(row.getFirst(), mapValue, row.getSecond())); + } + } + } + return res; + } + + private Multimap getJoinTable(List> rows) { + Multimap m = ArrayListMultimap.create(); + for (Pair v : rows) { + m.put(v.getFirst(), v.getSecond()); + } + return m; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/MapProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/MapProcessor.java new file mode 100644 index 00000000000..e9f89a9fc60 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/MapProcessor.java @@ -0,0 +1,33 @@ +/** + * 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.storm.streams.processors; + +import org.apache.storm.streams.operations.Function; + +public class MapProcessor extends BaseProcessor { + private final Function function; + + public MapProcessor(Function function) { + this.function = function; + } + + @Override + public void execute(T input) { + context.forward(function.apply(input)); + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/MapValuesProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/MapValuesProcessor.java new file mode 100644 index 00000000000..a963e2e48e2 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/MapValuesProcessor.java @@ -0,0 +1,34 @@ +/** + * 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.storm.streams.processors; + +import org.apache.storm.streams.Pair; +import org.apache.storm.streams.operations.Function; + +public class MapValuesProcessor extends BaseProcessor> { + private final Function function; + + public MapValuesProcessor(Function function) { + this.function = function; + } + + @Override + public void execute(Pair input) { + context.forward(Pair.of(input.getFirst(), function.apply(input.getSecond()))); + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/PeekProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/PeekProcessor.java new file mode 100644 index 00000000000..3014a98a023 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/PeekProcessor.java @@ -0,0 +1,34 @@ +/** + * 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.storm.streams.processors; + +import org.apache.storm.streams.operations.Consumer; + +public class PeekProcessor extends BaseProcessor { + private final Consumer consumer; + + public PeekProcessor(Consumer consumer) { + this.consumer = consumer; + } + + @Override + public void execute(T input) { + consumer.accept(input); + context.forward(input); + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/Processor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/Processor.java new file mode 100644 index 00000000000..5551682739c --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/Processor.java @@ -0,0 +1,51 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.streams.processors; + +import java.io.Serializable; + +/** + * A processor processes a stream of elements and produces some result. + * + * @param the type of the input that is processed + */ +public interface Processor extends Serializable { + /** + * Initializes the processor. This is typically invoked from the underlying + * storm bolt's prepare method. + * + * @param context the processor context + */ + void init(ProcessorContext context); + + /** + * Executes some operation on the input and possibly emits some result. + * + * @param input the input to be processed + * @param streamId the source stream id from where the input is received + */ + void execute(T input, String streamId); + + /** + * Punctuation marks end of a batch which can be used to compute and pass + * the results of one stage in the pipeline to the next. For e.g. emit the results of an aggregation. + * + * @param stream the stream id on which the punctuation arrived + */ + void punctuate(String stream); +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/ProcessorContext.java b/storm-core/src/jvm/org/apache/storm/streams/processors/ProcessorContext.java new file mode 100644 index 00000000000..2a10a224eb8 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/ProcessorContext.java @@ -0,0 +1,59 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.streams.processors; + +import java.io.Serializable; +import java.util.Set; + +/** + * Context information passed to the {@link Processor}. + */ +public interface ProcessorContext extends Serializable { + /** + * Forwards the input to all downstream processors. + * + * @param input the input + * @param the type of the input + */ + void forward(T input); + + /** + * Forwards the input to downstream processors at specified stream. + * + * @param input the input + * @param stream the stream to forward + * @param the type of the input + */ + void forward(T input, String stream); + + /** + * Returns if the processing is in a windowed context and should wait for + * punctuation before emitting results. + * + * @return if this is a windowed context or not + */ + boolean isWindowed(); + + /** + * Returns the windowed parent streams. These are the streams + * where punctuations arrive. + * + * @return the windowed parent streams + */ + Set getWindowedParentStreams(); +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceByKeyProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceByKeyProcessor.java new file mode 100644 index 00000000000..eba0de6a3fd --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceByKeyProcessor.java @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.streams.processors; + +import org.apache.storm.streams.Pair; +import org.apache.storm.streams.operations.Reducer; + +import java.util.HashMap; +import java.util.Map; + +public class ReduceByKeyProcessor extends BaseProcessor> implements BatchProcessor { + private final Reducer reducer; + private final Map state = new HashMap<>(); + + public ReduceByKeyProcessor(Reducer reducer) { + this.reducer = reducer; + } + + @Override + public void execute(Pair input) { + K key = input.getFirst(); + V val = input.getSecond(); + V agg = state.get(key); + agg = (agg == null) ? val : reducer.apply(agg, val); + state.put(key, agg); + mayBeForwardAggUpdate(Pair.of(key, agg)); + } + + @Override + public void finish() { + for (Map.Entry entry : state.entrySet()) { + context.forward(Pair.of(entry.getKey(), entry.getValue())); + } + state.clear(); + } + +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceProcessor.java new file mode 100644 index 00000000000..d64e1142036 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceProcessor.java @@ -0,0 +1,41 @@ +/** + * 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.storm.streams.processors; + +import org.apache.storm.streams.operations.Reducer; + +public class ReduceProcessor extends BaseProcessor implements BatchProcessor { + private final Reducer reducer; + private T agg; + + public ReduceProcessor(Reducer reducer) { + this.reducer = reducer; + } + + @Override + public void execute(T input) { + agg = (agg == null) ? input : reducer.apply(agg, input); + mayBeForwardAggUpdate(agg); + } + + @Override + public void finish() { + context.forward(agg); + agg = null; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/StateQueryProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/StateQueryProcessor.java new file mode 100644 index 00000000000..e577a65b53b --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/StateQueryProcessor.java @@ -0,0 +1,48 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.streams.processors; + +import org.apache.storm.state.KeyValueState; +import org.apache.storm.streams.Pair; +import org.apache.storm.streams.StreamState; + +public class StateQueryProcessor extends BaseProcessor implements StatefulProcessor { + private final StreamState streamState; + private KeyValueState keyValueState; + + public StateQueryProcessor(StreamState streamState) { + this.streamState = streamState; + } + + @Override + public void initState(KeyValueState keyValueState) { + this.keyValueState = keyValueState; + } + + public StreamState getStreamState() { + return streamState; + } + + @Override + protected void execute(K key) { + V value = keyValueState.get(key); + if (value != null) { + context.forward(Pair.of(key, value)); + } + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/StatefulProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/StatefulProcessor.java new file mode 100644 index 00000000000..acc4188a8d9 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/StatefulProcessor.java @@ -0,0 +1,36 @@ +/** + * 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.storm.streams.processors; + +import org.apache.storm.state.KeyValueState; + +/** + * Top level interface for processors that does stateful processing. + * + * @param the key type + * @param the value type + */ +public interface StatefulProcessor { + /** + * Initialize the state of the processor with the given + * {@link KeyValueState} + * + * @param keyValueState the key-value state + */ + void initState(KeyValueState keyValueState); +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/UpdateStateByKeyProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/UpdateStateByKeyProcessor.java new file mode 100644 index 00000000000..9e068a08ca3 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/UpdateStateByKeyProcessor.java @@ -0,0 +1,49 @@ +/** + * 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.storm.streams.processors; + +import org.apache.storm.state.KeyValueState; +import org.apache.storm.streams.Pair; +import org.apache.storm.streams.operations.Aggregator; + +public class UpdateStateByKeyProcessor extends BaseProcessor> implements StatefulProcessor { + private final Aggregator aggregator; + private KeyValueState keyValueState; + + public UpdateStateByKeyProcessor(Aggregator aggregator) { + this.aggregator = aggregator; + } + + @Override + public void initState(KeyValueState keyValueState) { + this.keyValueState = keyValueState; + } + + @Override + protected void execute(Pair input) { + K key = input.getFirst(); + V val = input.getSecond(); + R agg = keyValueState.get(key); + if (agg == null) { + agg = aggregator.init(); + } + R newAgg = aggregator.apply(val, agg); + keyValueState.put(key, newAgg); + context.forward(Pair.of(key, newAgg)); + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/windowing/BaseWindow.java b/storm-core/src/jvm/org/apache/storm/streams/windowing/BaseWindow.java new file mode 100644 index 00000000000..4526fa54549 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/windowing/BaseWindow.java @@ -0,0 +1,64 @@ +/** + * 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.storm.streams.windowing; + +import static org.apache.storm.topology.base.BaseWindowedBolt.Duration; + +public abstract class BaseWindow implements Window { + protected String timestampField; + protected String lateTupleStream; + protected Duration lag; + + @Override + public String getTimestampField() { + return timestampField; + } + + @Override + public String getLateTupleStream() { + return lateTupleStream; + } + + @Override + public Duration getLag() { + return lag; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + BaseWindow that = (BaseWindow) o; + + if (timestampField != null ? !timestampField.equals(that.timestampField) : that.timestampField != null) + return false; + if (lateTupleStream != null ? !lateTupleStream.equals(that.lateTupleStream) : that.lateTupleStream != null) + return false; + return lag != null ? lag.equals(that.lag) : that.lag == null; + + } + + @Override + public int hashCode() { + int result = timestampField != null ? timestampField.hashCode() : 0; + result = 31 * result + (lateTupleStream != null ? lateTupleStream.hashCode() : 0); + result = 31 * result + (lag != null ? lag.hashCode() : 0); + return result; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/windowing/SlidingWindows.java b/storm-core/src/jvm/org/apache/storm/streams/windowing/SlidingWindows.java new file mode 100644 index 00000000000..275d3c3c3a7 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/windowing/SlidingWindows.java @@ -0,0 +1,151 @@ +/** + * 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.storm.streams.windowing; + +import org.apache.storm.topology.base.BaseWindowedBolt; +import org.apache.storm.topology.base.BaseWindowedBolt.Count; +import org.apache.storm.topology.base.BaseWindowedBolt.Duration; + +/** + * A sliding window specification based on a window length and sliding interval. + * + * @param The type of the window length (e.g. Duration or Count) + * @param The type of the sliding interval (e.g. Duration or Count) + */ +public class SlidingWindows extends BaseWindow { + private final L windowLength; + private final I slidingInterval; + + private SlidingWindows(L windowLength, I slidingInterval) { + this.windowLength = windowLength; + this.slidingInterval = slidingInterval; + } + + /** + * {@inheritDoc} + */ + @Override + public L getWindowLength() { + return windowLength; + } + + /** + * {@inheritDoc} + */ + @Override + public I getSlidingInterval() { + return slidingInterval; + } + + /** + * Count based sliding window configuration. + * + * @param windowLength the number of tuples in the window + * @param slidingInterval the number of tuples after which the window slides + */ + public static SlidingWindows of(Count windowLength, Count slidingInterval) { + return new SlidingWindows<>(windowLength, slidingInterval); + } + + /** + * Time duration based sliding window configuration. + * + * @param windowLength the time duration of the window + * @param slidingInterval the time duration after which the window slides + */ + public static SlidingWindows of(Duration windowLength, Duration slidingInterval) { + return new SlidingWindows<>(windowLength, slidingInterval); + } + + /** + * Tuple count and time duration based sliding window configuration. + * + * @param windowLength the number of tuples in the window + * @param slidingInterval the time duration after which the window slides + */ + public static SlidingWindows of(Count windowLength, Duration slidingInterval) { + return new SlidingWindows<>(windowLength, slidingInterval); + } + + /** + * Time duration and count based sliding window configuration. + * + * @param windowLength the time duration of the window + * @param slidingInterval the number of tuples after which the window slides + */ + public static SlidingWindows of(Duration windowLength, Count slidingInterval) { + return new SlidingWindows<>(windowLength, slidingInterval); + } + + /** + * The name of the field in the tuple that contains the timestamp when the event occurred as a long value. + * This is used of event-time based processing. If this config is set and the field is not present in the incoming tuple, + * an {@link IllegalArgumentException} will be thrown. + * + * @param fieldName the name of the field that contains the timestamp + */ + public SlidingWindows withTimestampField(String fieldName) { + timestampField = fieldName; + return this; + } + + /** + * Specify a stream id on which late tuples are going to be emitted. They are going to be accessible via the + * {@link org.apache.storm.topology.WindowedBoltExecutor#LATE_TUPLE_FIELD} field. + * It must be defined on a per-component basis, and in conjunction with the + * {@link BaseWindowedBolt#withTimestampField}, otherwise {@link IllegalArgumentException} will be thrown. + * + * @param streamId the name of the stream used to emit late tuples on + */ + public SlidingWindows withLateTupleStream(String streamId) { + lateTupleStream = streamId; + return this; + } + + /** + * Specify the maximum time lag of the tuple timestamp in milliseconds. It means that the tuple timestamps + * cannot be out of order by more than this amount. + * + * @param duration the max lag duration + */ + public SlidingWindows withLag(Duration duration) { + lag = duration; + return this; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + if (!super.equals(o)) return false; + + SlidingWindows that = (SlidingWindows) o; + + if (windowLength != null ? !windowLength.equals(that.windowLength) : that.windowLength != null) return false; + return slidingInterval != null ? slidingInterval.equals(that.slidingInterval) : that.slidingInterval == null; + + } + + @Override + public int hashCode() { + int result = super.hashCode(); + result = 31 * result + (windowLength != null ? windowLength.hashCode() : 0); + result = 31 * result + (slidingInterval != null ? slidingInterval.hashCode() : 0); + return result; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/windowing/TumblingWindows.java b/storm-core/src/jvm/org/apache/storm/streams/windowing/TumblingWindows.java new file mode 100644 index 00000000000..71ed61710fd --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/windowing/TumblingWindows.java @@ -0,0 +1,119 @@ +/** + * 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.storm.streams.windowing; + +import org.apache.storm.topology.base.BaseWindowedBolt; + +import static org.apache.storm.topology.base.BaseWindowedBolt.Count; +import static org.apache.storm.topology.base.BaseWindowedBolt.Duration; + +/** + * A tumbling window specification. The window tumbles after the specified window length. + * + * @param the type of the length of the window (e.g Count, Duration) + */ +public class TumblingWindows extends BaseWindow { + private final L windowLength; + + private TumblingWindows(L windowLength) { + this.windowLength = windowLength; + } + + @Override + public L getWindowLength() { + return windowLength; + } + + @Override + public L getSlidingInterval() { + return windowLength; + } + + /** + * A count based tumbling window. + * + * @param count the number of tuples after which the window tumbles + */ + public static TumblingWindows of(Count count) { + return new TumblingWindows<>(count); + } + + /** + * A time duration based tumbling window. + * + * @param duration the time duration after which the window tumbles + */ + public static TumblingWindows of(Duration duration) { + return new TumblingWindows<>(duration); + } + + /** + * The name of the field in the tuple that contains the timestamp when the event occurred as a long value. + * This is used of event-time based processing. If this config is set and the field is not present in the incoming tuple, + * an {@link IllegalArgumentException} will be thrown. + * + * @param fieldName the name of the field that contains the timestamp + */ + public TumblingWindows withTimestampField(String fieldName) { + timestampField = fieldName; + return this; + } + + /** + * Specify a stream id on which late tuples are going to be emitted. They are going to be accessible via the + * {@link org.apache.storm.topology.WindowedBoltExecutor#LATE_TUPLE_FIELD} field. + * It must be defined on a per-component basis, and in conjunction with the + * {@link BaseWindowedBolt#withTimestampField}, otherwise {@link IllegalArgumentException} will be thrown. + * + * @param streamId the name of the stream used to emit late tuples on + */ + public TumblingWindows withLateTupleStream(String streamId) { + lateTupleStream = streamId; + return this; + } + + /** + * Specify the maximum time lag of the tuple timestamp in milliseconds. It means that the tuple timestamps + * cannot be out of order by more than this amount. + * + * @param duration the max lag duration + */ + public TumblingWindows withLag(Duration duration) { + lag = duration; + return this; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + if (!super.equals(o)) return false; + + TumblingWindows that = (TumblingWindows) o; + + return windowLength != null ? windowLength.equals(that.windowLength) : that.windowLength == null; + + } + + @Override + public int hashCode() { + int result = super.hashCode(); + result = 31 * result + (windowLength != null ? windowLength.hashCode() : 0); + return result; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/windowing/Window.java b/storm-core/src/jvm/org/apache/storm/streams/windowing/Window.java new file mode 100644 index 00000000000..40378e31542 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/windowing/Window.java @@ -0,0 +1,70 @@ +/** + * 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.storm.streams.windowing; + +import java.io.Serializable; + +import static org.apache.storm.topology.base.BaseWindowedBolt.Duration; + +/** + * The window specification within {@link org.apache.storm.streams.Stream}. + * + * @param the type of window length parameter (E.g. Count, Duration) + * @param the type of the sliding interval parameter (E.g. Count, Duration) + */ +public interface Window extends Serializable { + + /** + * The length of the window. + * + * @return the window length + */ + L getWindowLength(); + + /** + * The sliding interval of the window. + * + * @return the sliding interval + */ + I getSlidingInterval(); + + /** + * The name of the field in the tuple that contains the timestamp when the event occurred as a long value. + * This is used of event-time based processing. If this config is set and the field is not present in the incoming tuple, + * an {@link IllegalArgumentException} will be thrown. + * + * @return the timestamp field. + */ + String getTimestampField(); + + /** + * The name of the stream where late arriving tuples should be emitted. If this is not provided, + * the late tuples would be discarded. + * + * @return the name of the stream used to emit late tuples on + */ + String getLateTupleStream(); + + /** + * The maximum time lag of the tuple timestamp in milliseconds. It means that the tuple timestamps + * cannot be out of order by more than this amount. + * + * @return the lag + */ + Duration getLag(); +} diff --git a/storm-core/src/jvm/org/apache/storm/topology/StatefulWindowedBoltExecutor.java b/storm-core/src/jvm/org/apache/storm/topology/StatefulWindowedBoltExecutor.java index aff46e63ac8..2fc6f8d9e41 100644 --- a/storm-core/src/jvm/org/apache/storm/topology/StatefulWindowedBoltExecutor.java +++ b/storm-core/src/jvm/org/apache/storm/topology/StatefulWindowedBoltExecutor.java @@ -207,14 +207,14 @@ public void onExpiry(List events) { } @Override - public void onActivation(List events, List newEvents, List expired) { + public void onActivation(List events, List newEvents, List expired, Long timestamp) { if (isRecovering()) { String msg = String.format("Unexpected activation with events %s, newEvents %s, expired %s in recovering state. " + "recoveryStates %s ", events, newEvents, expired, recoveryStates); LOG.error(msg); throw new IllegalStateException(msg); } else { - parentListener.onActivation(events, newEvents, expired); + parentListener.onActivation(events, newEvents, expired, timestamp); updateWindowState(expired, newEvents); } } diff --git a/storm-core/src/jvm/org/apache/storm/topology/WindowedBoltExecutor.java b/storm-core/src/jvm/org/apache/storm/topology/WindowedBoltExecutor.java index fd98274ff45..b592e0b8066 100644 --- a/storm-core/src/jvm/org/apache/storm/topology/WindowedBoltExecutor.java +++ b/storm-core/src/jvm/org/apache/storm/topology/WindowedBoltExecutor.java @@ -327,9 +327,9 @@ public void onExpiry(List tuples) { } @Override - public void onActivation(List tuples, List newTuples, List expiredTuples) { + public void onActivation(List tuples, List newTuples, List expiredTuples, Long timestamp) { windowedOutputCollector.setContext(tuples); - bolt.execute(new TupleWindowImpl(tuples, newTuples, expiredTuples)); + bolt.execute(new TupleWindowImpl(tuples, newTuples, expiredTuples, timestamp)); } }; } diff --git a/storm-core/src/jvm/org/apache/storm/topology/base/BaseWindowedBolt.java b/storm-core/src/jvm/org/apache/storm/topology/base/BaseWindowedBolt.java index 2e2af0d4898..ba97c26c5b7 100644 --- a/storm-core/src/jvm/org/apache/storm/topology/base/BaseWindowedBolt.java +++ b/storm-core/src/jvm/org/apache/storm/topology/base/BaseWindowedBolt.java @@ -27,6 +27,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.Serializable; import java.util.HashMap; import java.util.Map; import java.util.concurrent.TimeUnit; @@ -40,7 +41,7 @@ public abstract class BaseWindowedBolt implements IWindowedBolt { /** * Holds a count value for count based windows and sliding intervals. */ - public static class Count { + public static class Count implements Serializable { public final int value; public Count(int value) { @@ -57,6 +58,22 @@ public static Count of(int value) { return new Count(value); } + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + Count count = (Count) o; + + return value == count.value; + + } + + @Override + public int hashCode() { + return value; + } + @Override public String toString() { return "Count{" + @@ -68,7 +85,7 @@ public String toString() { /** * Holds a Time duration for time based windows and sliding intervals. */ - public static class Duration { + public static class Duration implements Serializable { public final int value; public Duration(int value, TimeUnit timeUnit) { @@ -125,6 +142,22 @@ public static Duration seconds(int seconds) { return new Duration(seconds, TimeUnit.SECONDS); } + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + Duration duration = (Duration) o; + + return value == duration.value; + + } + + @Override + public int hashCode() { + return value; + } + @Override public String toString() { return "Duration{" + diff --git a/storm-core/src/jvm/org/apache/storm/trident/windowing/AbstractTridentWindowManager.java b/storm-core/src/jvm/org/apache/storm/trident/windowing/AbstractTridentWindowManager.java index f93527abd0c..a8fbb41cce4 100644 --- a/storm-core/src/jvm/org/apache/storm/trident/windowing/AbstractTridentWindowManager.java +++ b/storm-core/src/jvm/org/apache/storm/trident/windowing/AbstractTridentWindowManager.java @@ -121,7 +121,7 @@ public void onExpiry(List expiredEvents) { } @Override - public void onActivation(List events, List newEvents, List expired) { + public void onActivation(List events, List newEvents, List expired, Long timestamp) { LOG.debug("onActivation is invoked with events size: [{}]", events.size()); // trigger occurred, create an aggregation and keep them in store int currentTriggerId = triggerId.incrementAndGet(); diff --git a/storm-core/src/jvm/org/apache/storm/windowing/CountEvictionPolicy.java b/storm-core/src/jvm/org/apache/storm/windowing/CountEvictionPolicy.java index fb122027386..6a9a4f8d4dd 100644 --- a/storm-core/src/jvm/org/apache/storm/windowing/CountEvictionPolicy.java +++ b/storm-core/src/jvm/org/apache/storm/windowing/CountEvictionPolicy.java @@ -28,6 +28,7 @@ public class CountEvictionPolicy implements EvictionPolicy { protected final int threshold; protected final AtomicLong currentCount; + private EvictionContext context; public CountEvictionPolicy(int count) { this.threshold = count; @@ -62,7 +63,12 @@ public void track(Event event) { @Override public void setContext(EvictionContext context) { - // NOOP + this.context = context; + } + + @Override + public EvictionContext getContext() { + return context; } @Override diff --git a/storm-core/src/jvm/org/apache/storm/windowing/EvictionPolicy.java b/storm-core/src/jvm/org/apache/storm/windowing/EvictionPolicy.java index 05e4d93723d..774d0a3714b 100644 --- a/storm-core/src/jvm/org/apache/storm/windowing/EvictionPolicy.java +++ b/storm-core/src/jvm/org/apache/storm/windowing/EvictionPolicy.java @@ -72,4 +72,11 @@ enum Action { */ void setContext(EvictionContext context); + /** + * Returns the current context that is part of this eviction policy + * + * @return the eviction context + */ + EvictionContext getContext(); + } diff --git a/storm-core/src/jvm/org/apache/storm/windowing/TimeEvictionPolicy.java b/storm-core/src/jvm/org/apache/storm/windowing/TimeEvictionPolicy.java index e6462079ed2..802e6bb4291 100644 --- a/storm-core/src/jvm/org/apache/storm/windowing/TimeEvictionPolicy.java +++ b/storm-core/src/jvm/org/apache/storm/windowing/TimeEvictionPolicy.java @@ -22,11 +22,7 @@ */ public class TimeEvictionPolicy implements EvictionPolicy { private final int windowLength; - /** - * The reference time in millis for window calculations and - * expiring events. If not set it will default to System.currentTimeMillis() - */ - protected Long referenceTime; + protected EvictionContext evictionContext; /** * Constructs a TimeEvictionPolicy that evicts events older @@ -43,7 +39,7 @@ public TimeEvictionPolicy(int windowLength) { */ @Override public Action evict(Event event) { - long now = referenceTime == null ? System.currentTimeMillis() : referenceTime; + long now = evictionContext == null ? System.currentTimeMillis() : evictionContext.getReferenceTime(); long diff = now - event.getTimestamp(); if (diff >= windowLength) { return Action.EXPIRE; @@ -58,14 +54,19 @@ public void track(Event event) { @Override public void setContext(EvictionContext context) { - referenceTime = context.getReferenceTime(); + this.evictionContext = context; + } + + @Override + public EvictionContext getContext() { + return evictionContext; } @Override public String toString() { return "TimeEvictionPolicy{" + "windowLength=" + windowLength + - ", referenceTime=" + referenceTime + + ", evictionContext=" + evictionContext + '}'; } } diff --git a/storm-core/src/jvm/org/apache/storm/windowing/TupleWindowImpl.java b/storm-core/src/jvm/org/apache/storm/windowing/TupleWindowImpl.java index 2560d25ddcf..1e8b022673c 100644 --- a/storm-core/src/jvm/org/apache/storm/windowing/TupleWindowImpl.java +++ b/storm-core/src/jvm/org/apache/storm/windowing/TupleWindowImpl.java @@ -28,11 +28,17 @@ public class TupleWindowImpl implements TupleWindow { private final List tuples; private final List newTuples; private final List expiredTuples; + private final Long timestamp; public TupleWindowImpl(List tuples, List newTuples, List expiredTuples) { + this(tuples, newTuples, expiredTuples, null); + } + + public TupleWindowImpl(List tuples, List newTuples, List expiredTuples, Long timestamp) { this.tuples = tuples; this.newTuples = newTuples; this.expiredTuples = expiredTuples; + this.timestamp = timestamp; } @Override @@ -50,6 +56,11 @@ public List getExpired() { return expiredTuples; } + @Override + public Long getTimestamp() { + return timestamp != null ? timestamp : System.currentTimeMillis(); + } + @Override public String toString() { return "TupleWindowImpl{" + diff --git a/storm-core/src/jvm/org/apache/storm/windowing/WatermarkCountEvictionPolicy.java b/storm-core/src/jvm/org/apache/storm/windowing/WatermarkCountEvictionPolicy.java index 74240bb8c88..730436669b5 100644 --- a/storm-core/src/jvm/org/apache/storm/windowing/WatermarkCountEvictionPolicy.java +++ b/storm-core/src/jvm/org/apache/storm/windowing/WatermarkCountEvictionPolicy.java @@ -24,11 +24,6 @@ * @param the type of event tracked by this policy. */ public class WatermarkCountEvictionPolicy extends CountEvictionPolicy { - /* - * The reference time in millis for window calculations and - * expiring events. If not set it will default to System.currentTimeMillis() - */ - private long referenceTime; private long processed = 0L; public WatermarkCountEvictionPolicy(int count) { @@ -38,7 +33,7 @@ public WatermarkCountEvictionPolicy(int count) { @Override public Action evict(Event event) { Action action; - if (event.getTimestamp() <= referenceTime && processed < currentCount.get()) { + if (event.getTimestamp() <= super.getContext().getReferenceTime() && processed < currentCount.get()) { action = super.evict(event); if (action == Action.PROCESS) { ++processed; @@ -56,7 +51,7 @@ public void track(Event event) { @Override public void setContext(EvictionContext context) { - referenceTime = context.getReferenceTime(); + super.setContext(context); if (context.getCurrentCount() != null) { currentCount.set(context.getCurrentCount()); } else { @@ -68,7 +63,6 @@ public void setContext(EvictionContext context) { @Override public String toString() { return "WatermarkCountEvictionPolicy{" + - "referenceTime=" + referenceTime + "} " + super.toString(); } } diff --git a/storm-core/src/jvm/org/apache/storm/windowing/WatermarkTimeEvictionPolicy.java b/storm-core/src/jvm/org/apache/storm/windowing/WatermarkTimeEvictionPolicy.java index 53361d2a197..e5ecba4710d 100644 --- a/storm-core/src/jvm/org/apache/storm/windowing/WatermarkTimeEvictionPolicy.java +++ b/storm-core/src/jvm/org/apache/storm/windowing/WatermarkTimeEvictionPolicy.java @@ -45,7 +45,6 @@ public WatermarkTimeEvictionPolicy(int windowLength) { */ public WatermarkTimeEvictionPolicy(int windowLength, int lag) { super(windowLength); - referenceTime = 0L; this.lag = lag; } @@ -58,7 +57,8 @@ public WatermarkTimeEvictionPolicy(int windowLength, int lag) { */ @Override public Action evict(Event event) { - long diff = referenceTime - event.getTimestamp(); + long referenceTime = evictionContext.getReferenceTime() != null ? evictionContext.getReferenceTime() : 0L; + long diff = referenceTime - event.getTimestamp(); if (diff < -lag) { return Action.STOP; } else if (diff < 0) { diff --git a/storm-core/src/jvm/org/apache/storm/windowing/Window.java b/storm-core/src/jvm/org/apache/storm/windowing/Window.java index 83824480e7d..9a62eefb8b7 100644 --- a/storm-core/src/jvm/org/apache/storm/windowing/Window.java +++ b/storm-core/src/jvm/org/apache/storm/windowing/Window.java @@ -45,4 +45,11 @@ public interface Window { * @return the list of events expired from the window. */ List getExpired(); + + /** + * If processing based on event time, returns the watermark time otherwise the current timestamp. + * + * @return the window timestamp + */ + Long getTimestamp(); } diff --git a/storm-core/src/jvm/org/apache/storm/windowing/WindowLifecycleListener.java b/storm-core/src/jvm/org/apache/storm/windowing/WindowLifecycleListener.java index 03c0213a2f7..ea2c9974b27 100644 --- a/storm-core/src/jvm/org/apache/storm/windowing/WindowLifecycleListener.java +++ b/storm-core/src/jvm/org/apache/storm/windowing/WindowLifecycleListener.java @@ -37,6 +37,7 @@ public interface WindowLifecycleListener { * @param events the list of current events in the window. * @param newEvents the newly added events since last activation. * @param expired the expired events since last activation. + * @param referenceTime the reference (event or processing) time that resulted in activation */ - void onActivation(List events, List newEvents, List expired); + void onActivation(List events, List newEvents, List expired, Long referenceTime); } diff --git a/storm-core/src/jvm/org/apache/storm/windowing/WindowManager.java b/storm-core/src/jvm/org/apache/storm/windowing/WindowManager.java index 792509e047b..8021ba83729 100644 --- a/storm-core/src/jvm/org/apache/storm/windowing/WindowManager.java +++ b/storm-core/src/jvm/org/apache/storm/windowing/WindowManager.java @@ -142,7 +142,7 @@ public boolean onTrigger() { if (!events.isEmpty()) { prevWindowEvents.addAll(windowEvents); LOG.debug("invoking windowLifecycleListener onActivation, [{}] events in window.", events.size()); - windowLifecycleListener.onActivation(events, newEvents, expired); + windowLifecycleListener.onActivation(events, newEvents, expired, evictionPolicy.getContext().getReferenceTime()); } else { LOG.debug("No events in the window, skipping onActivation"); } diff --git a/storm-core/test/jvm/org/apache/storm/streams/ProcessorBoltTest.java b/storm-core/test/jvm/org/apache/storm/streams/ProcessorBoltTest.java new file mode 100644 index 00000000000..e9d51275d9f --- /dev/null +++ b/storm-core/test/jvm/org/apache/storm/streams/ProcessorBoltTest.java @@ -0,0 +1,165 @@ +/** + * 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.storm.streams; + +import com.google.common.collect.Multimap; +import org.apache.storm.streams.operations.aggregators.Sum; +import org.apache.storm.streams.processors.AggregateProcessor; +import org.apache.storm.streams.processors.FilterProcessor; +import org.apache.storm.streams.processors.Processor; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Tuple; +import org.apache.storm.tuple.Values; +import org.jgrapht.DirectedGraph; +import org.jgrapht.graph.DefaultDirectedGraph; +import org.junit.Before; +import org.junit.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Set; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; + +/** + * Unit tests for {@link ProcessorBolt} + */ +public class ProcessorBoltTest { + TopologyContext mockTopologyContext; + OutputCollector mockOutputCollector; + ProcessorBolt bolt; + Tuple mockTuple1; + Tuple mockTuple2; + Tuple mockTuple3; + Tuple punctuation; + Multimap mockStreamToProcessors; + DirectedGraph graph; + + @Before + public void setUp() throws Exception { + mockTopologyContext = Mockito.mock(TopologyContext.class); + mockOutputCollector = Mockito.mock(OutputCollector.class); + mockTuple1 = Mockito.mock(Tuple.class); + mockTuple2 = Mockito.mock(Tuple.class); + mockTuple3 = Mockito.mock(Tuple.class); + setUpMockTuples(mockTuple1, mockTuple2, mockTuple3); + punctuation = Mockito.mock(Tuple.class); + setUpPunctuation(punctuation); + mockStreamToProcessors = Mockito.mock(Multimap.class); + graph = new DefaultDirectedGraph(new StreamsEdgeFactory()); + + } + + @Test + public void testEmitAndAck() throws Exception { + setUpProcessorBolt(new FilterProcessor(x -> true)); + bolt.execute(mockTuple1); + ArgumentCaptor anchor = ArgumentCaptor.forClass(Collection.class); + ArgumentCaptor values = ArgumentCaptor.forClass(Values.class); + ArgumentCaptor os = ArgumentCaptor.forClass(String.class); + Mockito.verify(mockOutputCollector).emit(os.capture(), anchor.capture(), values.capture()); + assertEquals("outputstream", os.getValue()); + assertArrayEquals(new Object[]{mockTuple1}, anchor.getValue().toArray()); + assertEquals(new Values(100), values.getValue()); + Mockito.verify(mockOutputCollector, Mockito.times(1)).ack(mockTuple1); + } + + @Test + public void testAggResultAndAck() throws Exception { + setUpProcessorBolt(new AggregateProcessor<>(new Sum()), Collections.singleton("inputstream"), true, null); + bolt.execute(mockTuple2); + bolt.execute(mockTuple3); + bolt.execute(punctuation); + ArgumentCaptor anchor = ArgumentCaptor.forClass(Collection.class); + ArgumentCaptor values = ArgumentCaptor.forClass(Values.class); + ArgumentCaptor os = ArgumentCaptor.forClass(String.class); + Mockito.verify(mockOutputCollector, Mockito.times(2)).emit(os.capture(), anchor.capture(), values.capture()); + assertArrayEquals(new Object[]{mockTuple2, mockTuple3, punctuation}, anchor.getAllValues().get(0).toArray()); + assertArrayEquals(new Object[]{mockTuple2, mockTuple3, punctuation}, anchor.getAllValues().get(1).toArray()); + assertArrayEquals(new Object[]{new Values(200L), new Values("__punctuation")}, values.getAllValues().toArray()); + assertArrayEquals(new Object[]{"outputstream", "outputstream"}, os.getAllValues().toArray()); + Mockito.verify(mockOutputCollector).ack(mockTuple2); + Mockito.verify(mockOutputCollector).ack(mockTuple3); + Mockito.verify(mockOutputCollector).ack(punctuation); + } + + @Test + public void testEmitTs() throws Exception { + Tuple tupleWithTs = Mockito.mock(Tuple.class); + setUpMockTuples(tupleWithTs); + Mockito.when(tupleWithTs.getLongByField("ts")).thenReturn(12345L); + setUpProcessorBolt(new FilterProcessor(x -> true), "ts"); + bolt.execute(tupleWithTs); + ArgumentCaptor anchor = ArgumentCaptor.forClass(Collection.class); + ArgumentCaptor values = ArgumentCaptor.forClass(Values.class); + ArgumentCaptor os = ArgumentCaptor.forClass(String.class); + Mockito.verify(mockOutputCollector).emit(os.capture(), anchor.capture(), values.capture()); + assertEquals("outputstream", os.getValue()); + assertArrayEquals(new Object[]{tupleWithTs}, anchor.getValue().toArray()); + assertEquals(new Values(100, 12345L), values.getValue()); + Mockito.verify(mockOutputCollector, Mockito.times(1)).ack(tupleWithTs); + } + + private void setUpProcessorBolt(Processor processor) { + setUpProcessorBolt(processor, Collections.emptySet(), false, null); + } + + private void setUpProcessorBolt(Processor processor, String tsFieldName) { + setUpProcessorBolt(processor, Collections.emptySet(), false, tsFieldName); + } + + private void setUpProcessorBolt(Processor processor, + Set windowedParentStreams, + boolean isWindowed, + String tsFieldName) { + ProcessorNode node = new ProcessorNode(processor, "outputstream", new Fields("value")); + node.setWindowedParentStreams(windowedParentStreams); + node.setWindowed(isWindowed); + Mockito.when(mockStreamToProcessors.get(Mockito.anyString())).thenReturn(Collections.singletonList(node)); + graph.addVertex(node); + bolt = new ProcessorBolt("bolt1", graph, Collections.singletonList(node)); + if (tsFieldName != null && !tsFieldName.isEmpty()) { + bolt.setTimestampField(tsFieldName); + } + bolt.setStreamToInitialProcessors(mockStreamToProcessors); + bolt.prepare(new HashMap<>(), mockTopologyContext, mockOutputCollector); + } + + private void setUpMockTuples(Tuple... tuples) { + for (Tuple tuple : tuples) { + Mockito.when(tuple.size()).thenReturn(1); + Mockito.when(tuple.getValue(0)).thenReturn(100); + Mockito.when(tuple.getSourceComponent()).thenReturn("bolt0"); + Mockito.when(tuple.getSourceStreamId()).thenReturn("inputstream"); + } + } + + private void setUpPunctuation(Tuple punctuation) { + Mockito.when(punctuation.size()).thenReturn(1); + Mockito.when(punctuation.getValue(0)).thenReturn(WindowNode.PUNCTUATION); + Mockito.when(punctuation.getSourceComponent()).thenReturn("bolt0"); + Mockito.when(punctuation.getSourceStreamId()).thenReturn("inputstream"); + } +} diff --git a/storm-core/test/jvm/org/apache/storm/streams/StatefulProcessorBoltTest.java b/storm-core/test/jvm/org/apache/storm/streams/StatefulProcessorBoltTest.java new file mode 100644 index 00000000000..dbc7e27edf9 --- /dev/null +++ b/storm-core/test/jvm/org/apache/storm/streams/StatefulProcessorBoltTest.java @@ -0,0 +1,100 @@ +/** + * 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.storm.streams; + +import com.google.common.collect.Multimap; +import org.apache.storm.state.KeyValueState; +import org.apache.storm.streams.operations.aggregators.Count; +import org.apache.storm.streams.processors.Processor; +import org.apache.storm.streams.processors.UpdateStateByKeyProcessor; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Tuple; +import org.apache.storm.tuple.Values; +import org.jgrapht.DirectedGraph; +import org.jgrapht.graph.DefaultDirectedGraph; +import org.junit.Before; +import org.junit.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; + +/** + * Unit tests for {@link StatefulProcessorBolt} + */ +public class StatefulProcessorBoltTest { + TopologyContext mockTopologyContext; + OutputCollector mockOutputCollector; + StatefulProcessorBolt bolt; + Tuple mockTuple1; + DirectedGraph graph; + Multimap mockStreamToProcessors; + KeyValueState mockKeyValueState; + + @Before + public void setUp() throws Exception { + mockTopologyContext = Mockito.mock(TopologyContext.class); + mockOutputCollector = Mockito.mock(OutputCollector.class); + mockTuple1 = Mockito.mock(Tuple.class); + mockStreamToProcessors = Mockito.mock(Multimap.class); + mockKeyValueState = Mockito.mock(KeyValueState.class); + setUpMockTuples(mockTuple1); + } + + @Test + public void testEmitAndAck() throws Exception { + setUpStatefulProcessorBolt(new UpdateStateByKeyProcessor<>(new Count<>())); + bolt.execute(mockTuple1); + ArgumentCaptor anchor = ArgumentCaptor.forClass(Collection.class); + ArgumentCaptor values = ArgumentCaptor.forClass(Values.class); + ArgumentCaptor os = ArgumentCaptor.forClass(String.class); + Mockito.verify(mockOutputCollector).emit(os.capture(), anchor.capture(), values.capture()); + assertEquals("outputstream", os.getValue()); + assertArrayEquals(new Object[]{mockTuple1}, anchor.getValue().toArray()); + assertEquals(new Values("k", 1L), values.getValue()); + Mockito.verify(mockOutputCollector, Mockito.times(1)).ack(mockTuple1); + Mockito.verify(mockKeyValueState, Mockito.times(1)).put("k", 1L ); + } + + private void setUpStatefulProcessorBolt(Processor processor) { + ProcessorNode node = new ProcessorNode(processor, "outputstream", new Fields("value")); + Mockito.when(mockStreamToProcessors.get(Mockito.anyString())).thenReturn(Collections.singletonList(node)); + graph = new DefaultDirectedGraph(new StreamsEdgeFactory()); + graph.addVertex(node); + bolt = new StatefulProcessorBolt<>("bolt1", graph, Collections.singletonList(node)); + bolt.setStreamToInitialProcessors(mockStreamToProcessors); + bolt.prepare(new HashMap<>(), mockTopologyContext, mockOutputCollector); + bolt.initState(mockKeyValueState); + } + + private void setUpMockTuples(Tuple... tuples) { + for (Tuple tuple : tuples) { + Mockito.when(tuple.size()).thenReturn(1); + Mockito.when(tuple.getValue(0)).thenReturn(Pair.of("k", "v")); + Mockito.when(tuple.getSourceComponent()).thenReturn("bolt0"); + Mockito.when(tuple.getSourceStreamId()).thenReturn("inputstream"); + } + } +} diff --git a/storm-core/test/jvm/org/apache/storm/streams/StreamBuilderTest.java b/storm-core/test/jvm/org/apache/storm/streams/StreamBuilderTest.java new file mode 100644 index 00000000000..1498ae4bff6 --- /dev/null +++ b/storm-core/test/jvm/org/apache/storm/streams/StreamBuilderTest.java @@ -0,0 +1,219 @@ +/** + * 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.storm.streams; + +import org.apache.storm.generated.Bolt; +import org.apache.storm.generated.GlobalStreamId; +import org.apache.storm.generated.Grouping; +import org.apache.storm.generated.NullStruct; +import org.apache.storm.generated.SpoutSpec; +import org.apache.storm.generated.StormTopology; +import org.apache.storm.spout.SpoutOutputCollector; +import org.apache.storm.streams.operations.aggregators.Count; +import org.apache.storm.streams.operations.mappers.PairValueMapper; +import org.apache.storm.streams.operations.mappers.ValueMapper; +import org.apache.storm.streams.processors.BranchProcessor; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.IRichBolt; +import org.apache.storm.topology.IRichSpout; +import org.apache.storm.topology.OutputFieldsDeclarer; +import org.apache.storm.topology.base.BaseRichBolt; +import org.apache.storm.topology.base.BaseRichSpout; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Tuple; +import org.apache.storm.utils.Utils; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * Unit tests for {@link StreamBuilder} + */ +public class StreamBuilderTest { + StreamBuilder streamBuilder; + + @Before + public void setUp() throws Exception { + streamBuilder = new StreamBuilder(); + UniqueIdGen.getInstance().reset(); + } + + @Test(expected = IllegalArgumentException.class) + public void testSpoutNoDefaultStream() throws Exception { + Stream stream = streamBuilder.newStream(newSpout("test")); + stream.filter(x -> true); + streamBuilder.build(); + } + + @Test + public void testSpoutToBolt() throws Exception { + Stream stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID)); + stream.to(newBolt()); + StormTopology topology = streamBuilder.build(); + assertEquals(1, topology.get_spouts_size()); + assertEquals(1, topology.get_bolts_size()); + String spoutId = topology.get_spouts().keySet().iterator().next(); + Map expected = new HashMap<>(); + expected.put(new GlobalStreamId(spoutId, "default"), Grouping.shuffle(new NullStruct())); + assertEquals(expected, topology.get_bolts().values().iterator().next().get_common().get_inputs()); + } + + @Test + public void testBranch() throws Exception { + Stream stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID)); + Stream[] streams = stream.branch(x -> true); + StormTopology topology = streamBuilder.build(); + assertEquals(1, topology.get_spouts_size()); + assertEquals(1, topology.get_bolts_size()); + Map expected = new HashMap<>(); + String spoutId = topology.get_spouts().keySet().iterator().next(); + expected.put(new GlobalStreamId(spoutId, "default"), Grouping.shuffle(new NullStruct())); + assertEquals(expected, topology.get_bolts().values().iterator().next().get_common().get_inputs()); + assertEquals(1, streams.length); + assertEquals(1, streams[0].node.getOutputStreams().size()); + String parentStream = streams[0].node.getOutputStreams().iterator().next() + "-branch"; + assertEquals(1, streams[0].node.getParents(parentStream).size()); + Node processorNdoe = streams[0].node.getParents(parentStream).iterator().next(); + assertTrue(processorNdoe instanceof ProcessorNode); + assertTrue(((ProcessorNode) processorNdoe).getProcessor() instanceof BranchProcessor); + assertTrue(processorNdoe.getParents("default").iterator().next() instanceof SpoutNode); + } + + @Test + public void testJoin() throws Exception { + Stream stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new ValueMapper<>(0)); + Stream[] streams = stream.branch(x -> x % 2 == 0, x-> x % 3 == 0); + PairStream s1 = streams[0].mapToPair(x -> Pair.of(x, 1)); + PairStream s2 = streams[1].mapToPair(x -> Pair.of(x, 1)); + PairStream> sj = s1.join(s2); + assertEquals(Collections.singleton(s1.node), sj.node.getParents(s1.stream)); + assertEquals(Collections.singleton(s2.node), sj.node.getParents(s2.stream)); + } + + @Test + public void testGroupBy() throws Exception { + PairStream stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new PairValueMapper<>(0, 1)); + + stream.groupByKey().aggregateByKey(new Count<>()); + + StormTopology topology = streamBuilder.build(); + assertEquals(2, topology.get_bolts_size()); + Bolt bolt1 = topology.get_bolts().get("bolt1"); + Bolt bolt2 = topology.get_bolts().get("bolt2"); + assertEquals(Grouping.shuffle(new NullStruct()), bolt1.get_common().get_inputs().values().iterator().next()); + assertEquals(Grouping.fields(Collections.singletonList("key")), bolt2.get_common().get_inputs().values().iterator().next()); + } + + @Test + public void testGlobalAggregate() throws Exception { + Stream stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new ValueMapper<>(0)); + + stream.aggregate(new Count<>()); + + StormTopology topology = streamBuilder.build(); + assertEquals(2, topology.get_bolts_size()); + Bolt bolt1 = topology.get_bolts().get("bolt1"); + Bolt bolt2 = topology.get_bolts().get("bolt2"); + String spoutId = topology.get_spouts().keySet().iterator().next(); + Map expected1 = new HashMap<>(); + expected1.put(new GlobalStreamId(spoutId, "default"), Grouping.shuffle(new NullStruct())); + Map expected2 = new HashMap<>(); + expected2.put(new GlobalStreamId("bolt1", "s1"), Grouping.fields(Collections.emptyList())); + assertEquals(expected1, bolt1.get_common().get_inputs()); + assertEquals(expected2, bolt2.get_common().get_inputs()); + } + + @Test + public void testRepartition() throws Exception { + Stream stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new ValueMapper<>(0)); + stream.repartition(3).filter(x -> true).repartition(2).filter(x -> true).aggregate(new Count<>()); + StormTopology topology = streamBuilder.build(); + assertEquals(1, topology.get_spouts_size()); + SpoutSpec spout = topology.get_spouts().get("spout1"); + assertEquals(4, topology.get_bolts_size()); + Bolt bolt1 = topology.get_bolts().get("bolt1"); + Bolt bolt2 = topology.get_bolts().get("bolt2"); + Bolt bolt3 = topology.get_bolts().get("bolt3"); + Bolt bolt4 = topology.get_bolts().get("bolt4"); + assertEquals(1, spout.get_common().get_parallelism_hint()); + assertEquals(1, bolt1.get_common().get_parallelism_hint()); + assertEquals(3, bolt2.get_common().get_parallelism_hint()); + assertEquals(2, bolt3.get_common().get_parallelism_hint()); + assertEquals(2, bolt4.get_common().get_parallelism_hint()); + } + + @Test + public void testBranchAndJoin() throws Exception { + TopologyContext mockContext = Mockito.mock(TopologyContext.class); + OutputCollector mockCollector = Mockito.mock(OutputCollector.class); + Stream stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new ValueMapper<>(0)); + Stream[] streams = stream.branch(x -> x % 2 == 0, x -> x % 2 == 1); + PairStream> joined = streams[0].mapToPair(x -> Pair.of(x, 1)).join(streams[1].mapToPair(x -> Pair.of(x, 1))); + assertTrue(joined.getNode() instanceof ProcessorNode); + StormTopology topology = streamBuilder.build(); + assertEquals(1, topology.get_bolts_size()); + } + + private static IRichSpout newSpout(final String os) { + return new BaseRichSpout() { + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + declarer.declareStream(os, new Fields("value")); + } + + @Override + public void open(Map conf, TopologyContext context, SpoutOutputCollector collector) { + + } + + @Override + public void nextTuple() { + + } + }; + } + + private static IRichBolt newBolt() { + return new BaseRichBolt() { + + @Override + public void declareOutputFields(OutputFieldsDeclarer declarer) { + + } + + @Override + public void prepare(Map stormConf, TopologyContext context, OutputCollector collector) { + + } + + @Override + public void execute(Tuple input) { + + } + }; + } +} \ No newline at end of file diff --git a/storm-core/test/jvm/org/apache/storm/streams/WindowedProcessorBoltTest.java b/storm-core/test/jvm/org/apache/storm/streams/WindowedProcessorBoltTest.java new file mode 100644 index 00000000000..7428e3f66ca --- /dev/null +++ b/storm-core/test/jvm/org/apache/storm/streams/WindowedProcessorBoltTest.java @@ -0,0 +1,110 @@ +/** + * 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.storm.streams; + +import com.google.common.collect.Multimap; +import org.apache.storm.streams.operations.aggregators.Count; +import org.apache.storm.streams.processors.AggregateProcessor; +import org.apache.storm.streams.processors.Processor; +import org.apache.storm.streams.windowing.TumblingWindows; +import org.apache.storm.streams.windowing.Window; +import org.apache.storm.task.OutputCollector; +import org.apache.storm.task.TopologyContext; +import org.apache.storm.topology.base.BaseWindowedBolt; +import org.apache.storm.tuple.Fields; +import org.apache.storm.tuple.Tuple; +import org.apache.storm.tuple.Values; +import org.apache.storm.windowing.TupleWindow; +import org.jgrapht.DirectedGraph; +import org.jgrapht.graph.DefaultDirectedGraph; +import org.junit.Before; +import org.junit.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; + +import static org.junit.Assert.assertEquals; + +/** + * Unit test for {@link WindowedProcessorBolt} + */ +public class WindowedProcessorBoltTest { + TopologyContext mockTopologyContext; + OutputCollector mockOutputCollector; + WindowedProcessorBolt bolt; + Tuple mockTuple1; + Tuple mockTuple2; + Tuple mockTuple3; + DirectedGraph graph; + Multimap mockStreamToProcessors; + + @Before + public void setUp() throws Exception { + mockTopologyContext = Mockito.mock(TopologyContext.class); + mockOutputCollector = Mockito.mock(OutputCollector.class); + mockTuple1 = Mockito.mock(Tuple.class); + mockTuple2 = Mockito.mock(Tuple.class); + mockTuple3 = Mockito.mock(Tuple.class); + setUpMockTuples(mockTuple1, mockTuple2, mockTuple3); + mockStreamToProcessors = Mockito.mock(Multimap.class); + } + + @Test + public void testEmit() throws Exception { + Window window = TumblingWindows.of(BaseWindowedBolt.Count.of(2)); + setUpWindowedProcessorBolt(new AggregateProcessor<>(new Count<>()), window); + bolt.execute(getMockTupleWindow(mockTuple1, mockTuple2, mockTuple3)); + ArgumentCaptor values = ArgumentCaptor.forClass(Values.class); + ArgumentCaptor os = ArgumentCaptor.forClass(String.class); + Mockito.verify(mockOutputCollector, Mockito.times(2)).emit(os.capture(), values.capture()); + assertEquals("outputstream", os.getAllValues().get(0)); + assertEquals(new Values(3L), values.getAllValues().get(0)); + assertEquals("outputstream", os.getAllValues().get(1)); + assertEquals(new Values(WindowNode.PUNCTUATION), values.getAllValues().get(1)); + } + + private void setUpWindowedProcessorBolt(Processor processor, Window window) { + ProcessorNode node = new ProcessorNode(processor, "outputstream", new Fields("value")); + node.setWindowed(true); + Mockito.when(mockStreamToProcessors.get(Mockito.anyString())).thenReturn(Collections.singletonList(node)); + Mockito.when(mockStreamToProcessors.keySet()).thenReturn(Collections.singleton("inputstream")); + graph = new DefaultDirectedGraph<>(new StreamsEdgeFactory()); + graph.addVertex(node); + bolt = new WindowedProcessorBolt("bolt1", graph, Collections.singletonList(node), window); + bolt.setStreamToInitialProcessors(mockStreamToProcessors); + bolt.prepare(new HashMap<>(), mockTopologyContext, mockOutputCollector); + } + + private void setUpMockTuples(Tuple... tuples) { + for (Tuple tuple : tuples) { + Mockito.when(tuple.size()).thenReturn(1); + Mockito.when(tuple.getValue(0)).thenReturn(100); + Mockito.when(tuple.getSourceComponent()).thenReturn("bolt0"); + Mockito.when(tuple.getSourceStreamId()).thenReturn("inputstream"); + } + } + + private TupleWindow getMockTupleWindow(Tuple... tuples) { + TupleWindow tupleWindow = Mockito.mock(TupleWindow.class); + Mockito.when(tupleWindow.get()).thenReturn(Arrays.asList(tuples)); + return tupleWindow; + } +} diff --git a/storm-core/test/jvm/org/apache/storm/windowing/WindowManagerTest.java b/storm-core/test/jvm/org/apache/storm/windowing/WindowManagerTest.java index 66455662113..6c170c64b33 100644 --- a/storm-core/test/jvm/org/apache/storm/windowing/WindowManagerTest.java +++ b/storm-core/test/jvm/org/apache/storm/windowing/WindowManagerTest.java @@ -60,7 +60,7 @@ public void onExpiry(List events) { } @Override - public void onActivation(List events, List newEvents, List expired) { + public void onActivation(List events, List newEvents, List expired, Long timestamp) { onActivationEvents = events; allOnActivationEvents.add(events); onActivationNewEvents = newEvents; From 3a10865c628fa0606456826e28ce8838baf60134 Mon Sep 17 00:00:00 2001 From: Arun Mahadevan Date: Tue, 25 Oct 2016 01:13:53 +0530 Subject: [PATCH 02/29] [STORM-1961] A few fixes and refactoring 1. Added typed tuples 2. Change groupByKey semantics and refactor examples 3. Handle punctuations correctly 4. Added countByKey and count 5. Added left, right and full outer joins 6. Per partition combine for aggregate/reduce --- .../starter/streams/AggregateExample.java | 96 +++++ .../starter/streams/StateQueryExample.java | 4 +- .../starter/streams/StatefulWordCount.java | 16 +- .../starter/streams/TypedTupleExample.java | 64 ++++ .../starter/streams/WindowedWordCount.java | 18 +- .../starter/streams/WordCountToBolt.java | 7 +- .../apache/storm/streams/GroupingInfo.java | 16 + .../jvm/org/apache/storm/streams/Node.java | 53 ++- .../jvm/org/apache/storm/streams/Pair.java | 19 + .../org/apache/storm/streams/PairStream.java | 345 ++++++++++++++++-- .../apache/storm/streams/PartitionNode.java | 7 +- .../storm/streams/ProcessorBoltDelegate.java | 111 ++++-- .../apache/storm/streams/ProcessorNode.java | 24 +- .../jvm/org/apache/storm/streams/Stream.java | 177 +++++++-- .../apache/storm/streams/StreamBuilder.java | 130 ++++--- .../org/apache/storm/streams/StreamUtil.java | 25 +- .../org/apache/storm/streams/WindowNode.java | 1 + .../{Aggregator.java => BiFunction.java} | 25 +- .../operations/CombinerAggregator.java | 97 +++++ .../storm/streams/operations/Reducer.java | 2 +- .../streams/operations/StateUpdater.java | 67 ++++ .../streams/operations/aggregators/Count.java | 18 +- .../streams/operations/aggregators/Sum.java | 16 +- .../operations/mappers/TupleValueMappers.java | 174 +++++++++ .../processors/AggregateByKeyProcessor.java | 47 ++- .../processors/AggregateProcessor.java | 44 ++- .../streams/processors/BranchProcessor.java | 6 +- .../processors/EmittingProcessorContext.java | 35 +- .../streams/processors/JoinProcessor.java | 54 ++- .../MergeAggregateByKeyProcessor.java | 54 +++ .../processors/MergeAggregateProcessor.java | 47 +++ .../streams/processors/ReduceProcessor.java | 6 +- .../processors/UpdateStateByKeyProcessor.java | 12 +- .../apache/storm/streams/tuple/Tuple10.java | 112 ++++++ .../storm/streams/{ => tuple}/Tuple3.java | 27 +- .../apache/storm/streams/tuple/Tuple4.java | 76 ++++ .../apache/storm/streams/tuple/Tuple5.java | 82 +++++ .../apache/storm/streams/tuple/Tuple6.java | 89 +++++ .../apache/storm/streams/tuple/Tuple7.java | 94 +++++ .../apache/storm/streams/tuple/Tuple8.java | 100 +++++ .../apache/storm/streams/tuple/Tuple9.java | 106 ++++++ .../storm/streams/ProcessorBoltTest.java | 13 +- .../streams/StatefulProcessorBoltTest.java | 15 +- .../storm/streams/StreamBuilderTest.java | 57 ++- .../streams/WindowedProcessorBoltTest.java | 2 +- .../streams/processors/JoinProcessorTest.java | 108 ++++++ 46 files changed, 2398 insertions(+), 300 deletions(-) create mode 100644 examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java create mode 100644 examples/storm-starter/src/jvm/org/apache/storm/starter/streams/TypedTupleExample.java rename storm-core/src/jvm/org/apache/storm/streams/operations/{Aggregator.java => BiFunction.java} (62%) create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/CombinerAggregator.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/StateUpdater.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/operations/mappers/TupleValueMappers.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateByKeyProcessor.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateProcessor.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple10.java rename storm-core/src/jvm/org/apache/storm/streams/{ => tuple}/Tuple3.java (63%) create mode 100644 storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple4.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple5.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple6.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple7.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple8.java create mode 100644 storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple9.java create mode 100644 storm-core/test/jvm/org/apache/storm/streams/processors/JoinProcessorTest.java diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java new file mode 100644 index 00000000000..91dfadb2df1 --- /dev/null +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java @@ -0,0 +1,96 @@ +/** + * 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.storm.starter.streams; + +import org.apache.storm.Config; +import org.apache.storm.LocalCluster; +import org.apache.storm.StormSubmitter; +import org.apache.storm.starter.spout.RandomIntegerSpout; +import org.apache.storm.streams.Pair; +import org.apache.storm.streams.PairStream; +import org.apache.storm.streams.Stream; +import org.apache.storm.streams.StreamBuilder; +import org.apache.storm.streams.operations.CombinerAggregator; +import org.apache.storm.streams.operations.mappers.TupleValueMapper; +import org.apache.storm.streams.operations.mappers.TupleValueMappers; +import org.apache.storm.streams.operations.mappers.ValueMapper; +import org.apache.storm.streams.tuple.Tuple3; +import org.apache.storm.streams.windowing.TumblingWindows; +import org.apache.storm.topology.base.BaseWindowedBolt; +import org.apache.storm.trident.windowing.config.TumblingDurationWindow; +import org.apache.storm.utils.Utils; + +import static org.apache.storm.topology.base.BaseWindowedBolt.Count; + +/** + * An example that illustrates the global aggregate + */ +public class AggregateExample { + @SuppressWarnings("unchecked") + public static void main(String[] args) throws Exception { + StreamBuilder builder = new StreamBuilder(); + /** + * Computes average of the stream of numbers emitted by the spout. Internally the per-partition + * sum and counts are accumulated and emitted to a downstream task where the partially accumulated + * results are merged and the final result is emitted. + */ + builder.newStream(new RandomIntegerSpout(), new ValueMapper(0), 2) + .window(TumblingWindows.of(BaseWindowedBolt.Duration.seconds(5))) + .filter(x -> x > 0 && x < 500) + .aggregate(new Avg()) + .print(); + + Config config = new Config(); + if (args.length > 0) { + config.setNumWorkers(1); + StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build()); + } else { + LocalCluster cluster = new LocalCluster(); + cluster.submitTopology("test", config, builder.build()); + Utils.sleep(60000); + cluster.killTopology("test"); + cluster.shutdown(); + } + } + + private static class Avg implements CombinerAggregator, Double> { + @Override + public Pair init() { + return Pair.of(0, 0); + } + + @Override + public Pair apply(Pair sumAndCount, Integer value) { + return Pair.of(sumAndCount.getFirst() + value, sumAndCount.getSecond() + 1); + } + + @Override + public Pair merge(Pair sumAndCount1, Pair sumAndCount2) { + System.out.println("Merge " + sumAndCount1 + " and " + sumAndCount2); + return Pair.of( + sumAndCount1.getFirst() + sumAndCount2.getFirst(), + sumAndCount1.getSecond() + sumAndCount2.getSecond() + ); + } + + @Override + public Double result(Pair sumAndCount) { + return (double) sumAndCount.getFirst()/sumAndCount.getSecond(); + } + } +} diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java index e76dd3c0fc6..2f0a4a3eec4 100644 --- a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java @@ -25,7 +25,6 @@ import org.apache.storm.streams.Stream; import org.apache.storm.streams.StreamBuilder; import org.apache.storm.streams.StreamState; -import org.apache.storm.streams.operations.aggregators.Count; import org.apache.storm.streams.operations.mappers.ValueMapper; import org.apache.storm.task.TopologyContext; import org.apache.storm.testing.TestWordSpout; @@ -56,8 +55,7 @@ public static void main(String[] args) throws Exception { StreamBuilder builder = new StreamBuilder(); StreamState ss = builder.newStream(new TestWordSpout(), new ValueMapper(0)) .mapToPair(w -> Pair.of(w, 1)) - .groupByKey() - .updateStateByKey(new Count<>()); + .updateStateByKey(0L, (count, val) -> count + 1); /* * A stream of words emitted by the QuerySpout is used as diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StatefulWordCount.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StatefulWordCount.java index f6ae6b07496..ce7470dfab1 100644 --- a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StatefulWordCount.java +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StatefulWordCount.java @@ -23,14 +23,15 @@ import org.apache.storm.streams.Pair; import org.apache.storm.streams.PairStream; import org.apache.storm.streams.StreamBuilder; -import org.apache.storm.streams.operations.Aggregator; -import org.apache.storm.streams.operations.aggregators.Count; +import org.apache.storm.streams.operations.StateUpdater; import org.apache.storm.streams.operations.mappers.ValueMapper; +import org.apache.storm.streams.windowing.TumblingWindows; import org.apache.storm.testing.TestWordSpout; +import org.apache.storm.topology.base.BaseWindowedBolt; import org.apache.storm.utils.Utils; /** - * A stateful word count that uses {@link PairStream#updateStateByKey(Aggregator)} to + * A stateful word count that uses {@link PairStream#updateStateByKey(StateUpdater)} to * save the counts in a key value state. This example uses Redis state store. *

* You should start a local redis instance before running the 'storm jar' command. By default @@ -48,19 +49,20 @@ public class StatefulWordCount { public static void main(String[] args) throws Exception { StreamBuilder builder = new StreamBuilder(); // a stream of words - builder.newStream(new TestWordSpout(), new ValueMapper(0)) + builder.newStream(new TestWordSpout(), new ValueMapper(0), 2) + .window(TumblingWindows.of(BaseWindowedBolt.Duration.seconds(2))) /* * create a stream of (word, 1) pairs */ .mapToPair(w -> Pair.of(w, 1)) /* - * group by the word + * compute the word counts in the last two second window */ - .groupByKey() + .countByKey() /* * update the word counts in the state */ - .updateStateByKey(new Count<>()) + .updateStateByKey(0L, (x, y) -> x + y) /* * convert the state back to a stream and print the results */ diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/TypedTupleExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/TypedTupleExample.java new file mode 100644 index 00000000000..193ad661191 --- /dev/null +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/TypedTupleExample.java @@ -0,0 +1,64 @@ +/** + * 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.storm.starter.streams; + +import org.apache.storm.Config; +import org.apache.storm.LocalCluster; +import org.apache.storm.StormSubmitter; +import org.apache.storm.starter.spout.RandomIntegerSpout; +import org.apache.storm.streams.Pair; +import org.apache.storm.streams.PairStream; +import org.apache.storm.streams.Stream; +import org.apache.storm.streams.StreamBuilder; +import org.apache.storm.streams.operations.mappers.TupleValueMappers; +import org.apache.storm.streams.tuple.Tuple3; +import org.apache.storm.streams.windowing.TumblingWindows; +import org.apache.storm.utils.Utils; + +import static org.apache.storm.topology.base.BaseWindowedBolt.Count; + +/** + * An example that illustrates the usage of typed tuples (TupleN<..>) and {@link TupleValueMappers}. + */ +public class TypedTupleExample { + @SuppressWarnings("unchecked") + public static void main(String[] args) throws Exception { + StreamBuilder builder = new StreamBuilder(); + /** + * The spout emits sequences of (Integer, Long, Long). TupleValueMapper can be used to extract fields + * from the values and produce a stream of typed tuple (Tuple3 in this case. + */ + Stream> stream = builder.newStream(new RandomIntegerSpout(), TupleValueMappers.of(0, 1, 2)); + + PairStream pairs = stream.mapToPair(t -> Pair.of(t._2 / 10000, t._1)); + + pairs.window(TumblingWindows.of(Count.of(10))).groupByKey().print(); + + Config config = new Config(); + if (args.length > 0) { + config.setNumWorkers(1); + StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build()); + } else { + LocalCluster cluster = new LocalCluster(); + cluster.submitTopology("test", config, builder.build()); + Utils.sleep(60000); + cluster.killTopology("test"); + cluster.shutdown(); + } + } +} diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WindowedWordCount.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WindowedWordCount.java index c6e2f4a1491..0765a741dbf 100644 --- a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WindowedWordCount.java +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WindowedWordCount.java @@ -23,7 +23,6 @@ import org.apache.storm.starter.spout.RandomSentenceSpout; import org.apache.storm.streams.Pair; import org.apache.storm.streams.StreamBuilder; -import org.apache.storm.streams.operations.aggregators.Count; import org.apache.storm.streams.operations.mappers.ValueMapper; import org.apache.storm.streams.windowing.TumblingWindows; import org.apache.storm.utils.Utils; @@ -39,12 +38,11 @@ public class WindowedWordCount { public static void main(String[] args) throws Exception { StreamBuilder builder = new StreamBuilder(); // A stream of random sentences - builder.newStream(new RandomSentenceSpout(), new ValueMapper(0)) + builder.newStream(new RandomSentenceSpout(), new ValueMapper(0), 2) /* - * Increase the parallelism of this stream. Further operations - * on this stream will execute at this level of parallelism. + * a two seconds tumbling window */ - .repartition(2) + .window(TumblingWindows.of(Duration.seconds(2))) /* * split the sentences to words */ @@ -53,18 +51,10 @@ public static void main(String[] args) throws Exception { * create a stream of (word, 1) pairs */ .mapToPair(w -> Pair.of(w, 1)) - /* - * group by word so that the same words end up in the same partition - */ - .groupByKey() - /* - * a two seconds tumbling window - */ - .window(TumblingWindows.of(Duration.seconds(2))) /* * compute the word counts in the last two second window */ - .aggregateByKey(new Count<>()) + .countByKey() /* * emit the count for the words that occurred * at-least five times in the last two seconds diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WordCountToBolt.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WordCountToBolt.java index a7116963d37..dd7923a8bdb 100644 --- a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WordCountToBolt.java +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WordCountToBolt.java @@ -26,7 +26,6 @@ import org.apache.storm.redis.common.mapper.RedisStoreMapper; import org.apache.storm.streams.Pair; import org.apache.storm.streams.StreamBuilder; -import org.apache.storm.streams.operations.aggregators.Count; import org.apache.storm.streams.operations.mappers.ValueMapper; import org.apache.storm.testing.TestWordSpout; import org.apache.storm.topology.IRichBolt; @@ -55,14 +54,10 @@ public static void main(String[] args) throws Exception { * create a stream of (word, 1) pairs */ .mapToPair(w -> Pair.of(w, 1)) - /* - * group by key (word) - */ - .groupByKey() /* * aggregate the count */ - .aggregateByKey(new Count<>()) + .countByKey() /* * The result of aggregation is forwarded to * the RedisStoreBolt. The forwarded tuple is a diff --git a/storm-core/src/jvm/org/apache/storm/streams/GroupingInfo.java b/storm-core/src/jvm/org/apache/storm/streams/GroupingInfo.java index 81def4b0580..9e8c893c564 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/GroupingInfo.java +++ b/storm-core/src/jvm/org/apache/storm/streams/GroupingInfo.java @@ -75,6 +75,22 @@ public Fields getFields() { return fields; } + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + GroupingInfo that = (GroupingInfo) o; + + return fields != null ? fields.equals(that.fields) : that.fields == null; + + } + + @Override + public int hashCode() { + return fields != null ? fields.hashCode() : 0; + } + @Override public String toString() { return "GroupingInfo{" + diff --git a/storm-core/src/jvm/org/apache/storm/streams/Node.java b/storm-core/src/jvm/org/apache/storm/streams/Node.java index f9de390a9c9..3507f508227 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/Node.java +++ b/storm-core/src/jvm/org/apache/storm/streams/Node.java @@ -38,30 +38,47 @@ abstract class Node implements Serializable { private final Set outputStreams; protected final Fields outputFields; + protected GroupingInfo groupingInfo; protected String componentId; protected int parallelism; // the parent streams that this node subscribes to private final Multimap parentStreams = ArrayListMultimap.create(); + private boolean windowed; + private boolean emitsPair; - Node(Set outputStreams, Fields outputFields, String componentId, int parallelism) { + Node(Set outputStreams, Fields outputFields, String componentId, int parallelism, + GroupingInfo groupingInfo) { this.outputStreams = new HashSet<>(outputStreams); this.outputFields = outputFields; this.componentId = componentId; this.parallelism = parallelism; + this.groupingInfo = groupingInfo; } - Node(String outputStream, Fields outputFields, String componentId, int parallelism) { - this(Collections.singleton(outputStream), outputFields, componentId, parallelism); + Node(String outputStream, Fields outputFields, String componentId, int parallelism, GroupingInfo groupingInfo) { + this(Collections.singleton(outputStream), outputFields, componentId, parallelism, groupingInfo); } - Node(String outputStream, Fields outputFields, String componentId) { - this(outputStream, outputFields, componentId, 1); + Node(String outputStream, Fields outputFields, String componentId, GroupingInfo groupingInfo) { + this(outputStream, outputFields, componentId, 1, groupingInfo); } Node(String outputStream, Fields outputFields) { this(outputStream, outputFields, null); } + Node(String outputStream, Fields outputFields, GroupingInfo groupingInfo) { + this(outputStream, outputFields, null, groupingInfo); + } + + GroupingInfo getGroupingInfo() { + return groupingInfo; + } + + void setGroupingInfo(GroupingInfo groupingInfo) { + this.groupingInfo = groupingInfo; + } + public Fields getOutputFields() { return outputFields; } @@ -94,6 +111,14 @@ Set getOutputStreams() { return Collections.unmodifiableSet(outputStreams); } + public boolean isWindowed() { + return windowed; + } + + public void setWindowed(boolean windowed) { + this.windowed = windowed; + } + Collection getParentStreams(Node parent) { return parentStreams.get(parent); } @@ -103,6 +128,10 @@ Set getParents(String stream) { return new HashSet<>(rev.get(stream)); } + Set getParents() { + return parentStreams.keySet(); + } + void addOutputStream(String streamId) { outputStreams.add(streamId); } @@ -117,13 +146,25 @@ static Fields getOutputFields(IComponent component, String streamId) { return new Fields(); } + public boolean emitsPair() { + return emitsPair; + } + + public void setEmitsPair(boolean emitsPair) { + this.emitsPair = emitsPair; + } + @Override public String toString() { return "Node{" + - "outputStreams='" + outputStreams + '\'' + + "outputStreams=" + outputStreams + ", outputFields=" + outputFields + + ", groupingInfo=" + groupingInfo + ", componentId='" + componentId + '\'' + ", parallelism=" + parallelism + + ", parentStreams=" + parentStreams + + ", windowed=" + windowed + + ", emitsPair=" + emitsPair + '}'; } } diff --git a/storm-core/src/jvm/org/apache/storm/streams/Pair.java b/storm-core/src/jvm/org/apache/storm/streams/Pair.java index 00443598510..e5eb792ecd8 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/Pair.java +++ b/storm-core/src/jvm/org/apache/storm/streams/Pair.java @@ -71,6 +71,25 @@ public static Pair of(T1 first, T2 second) { return new Pair<>(first, second); } + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + Pair pair = (Pair) o; + + if (first != null ? !first.equals(pair.first) : pair.first != null) return false; + return second != null ? second.equals(pair.second) : pair.second == null; + + } + + @Override + public int hashCode() { + int result = first != null ? first.hashCode() : 0; + result = 31 * result + (second != null ? second.hashCode() : 0); + return result; + } + @Override public String toString() { return "(" + first + ", " + second + ')'; diff --git a/storm-core/src/jvm/org/apache/storm/streams/PairStream.java b/storm-core/src/jvm/org/apache/storm/streams/PairStream.java index 2d18b30536f..964cdbaff53 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/PairStream.java +++ b/storm-core/src/jvm/org/apache/storm/streams/PairStream.java @@ -18,26 +18,31 @@ package org.apache.storm.streams; import org.apache.storm.Config; -import org.apache.storm.streams.operations.Aggregator; +import org.apache.storm.streams.operations.BiFunction; +import org.apache.storm.streams.operations.CombinerAggregator; import org.apache.storm.streams.operations.Consumer; import org.apache.storm.streams.operations.FlatMapFunction; import org.apache.storm.streams.operations.Function; import org.apache.storm.streams.operations.PairValueJoiner; import org.apache.storm.streams.operations.Predicate; import org.apache.storm.streams.operations.Reducer; +import org.apache.storm.streams.operations.StateUpdater; import org.apache.storm.streams.operations.ValueJoiner; +import org.apache.storm.streams.operations.aggregators.Count; import org.apache.storm.streams.processors.AggregateByKeyProcessor; import org.apache.storm.streams.processors.FlatMapValuesProcessor; import org.apache.storm.streams.processors.JoinProcessor; import org.apache.storm.streams.processors.MapValuesProcessor; +import org.apache.storm.streams.processors.MergeAggregateByKeyProcessor; import org.apache.storm.streams.processors.ReduceByKeyProcessor; import org.apache.storm.streams.processors.UpdateStateByKeyProcessor; import org.apache.storm.streams.windowing.Window; import org.apache.storm.tuple.Fields; import java.util.ArrayList; -import java.util.Collection; import java.util.List; +import java.util.Optional; +import java.util.Set; /** * Represents a stream of key-value pairs. @@ -49,6 +54,7 @@ public class PairStream extends Stream> { PairStream(StreamBuilder topology, Node node) { super(topology, node); + node.setEmitsPair(true); } /** @@ -60,7 +66,8 @@ public class PairStream extends Stream> { * @return the new stream */ public PairStream mapValues(Function function) { - return new PairStream<>(streamBuilder, addProcessorNode(new MapValuesProcessor<>(function), KEY_VALUE)); + return new PairStream<>(streamBuilder, + addProcessorNode(new MapValuesProcessor<>(function), KEY_VALUE, true)); } /** @@ -71,19 +78,45 @@ public PairStream mapValues(Function function) * @param the result type * @return the new stream */ - public PairStream flatMapValues(FlatMapFunction function) { - return new PairStream<>(streamBuilder, addProcessorNode(new FlatMapValuesProcessor<>(function), KEY_VALUE)); + public PairStream flatMapValues(FlatMapFunction function) { + return new PairStream<>(streamBuilder, + addProcessorNode(new FlatMapValuesProcessor<>(function), KEY_VALUE, true)); } /** - * Aggregates the values for each key of this stream using the given {@link Aggregator}. + * Aggregates the values for each key of this stream using the given initial value, accumulator and combiner. * - * @param aggregator the aggregator + * @param initialValue the initial value of the result + * @param accumulator the accumulator + * @param combiner the combiner + * @param the result type + * @return the new stream + */ + public PairStream aggregateByKey(R initialValue, + BiFunction accumulator, + BiFunction combiner) { + return combineByKey(CombinerAggregator.of(initialValue, accumulator, combiner)); + } + + /** + * Aggregates the values for each key of this stream using the given {@link CombinerAggregator}. + * + * @param aggregator the combiner aggregator + * @param the accumulator type * @param the result type * @return the new stream */ - public PairStream aggregateByKey(Aggregator aggregator) { - return new PairStream<>(streamBuilder, addProcessorNode(new AggregateByKeyProcessor<>(aggregator), KEY_VALUE)); + public PairStream aggregateByKey(CombinerAggregator aggregator) { + return combineByKey(aggregator); + } + + /** + * Counts the values for each key of this stream. + * + * @return the new stream + */ + public PairStream countByKey() { + return aggregateByKey(new Count<>()); } /** @@ -93,7 +126,7 @@ public PairStream aggregateByKey(Aggregator ag * @return the new stream */ public PairStream reduceByKey(Reducer reducer) { - return new PairStream<>(streamBuilder, addProcessorNode(new ReduceByKeyProcessor<>(reducer), KEY_VALUE)); + return combineByKey(reducer); } /** @@ -101,8 +134,8 @@ public PairStream reduceByKey(Reducer reducer) { * * @return the new stream */ - public PairStream groupByKey() { - return partitionBy(KEY); + public PairStream> groupByKey() { + return partitionByKey().aggregatePartition(new MergeValues<>()); } /** @@ -114,7 +147,7 @@ public PairStream groupByKey() { * @return the new stream */ public PairStream> groupByKeyAndWindow(Window window) { - return groupByKey().window(window).aggregateByKey(new MergeValues<>()); + return partitionByKey().window(window).aggregatePartition(new MergeValues<>()); } /** @@ -126,7 +159,7 @@ public PairStream> groupByKeyAndWindow(Window window) { * @return the new stream */ public PairStream reduceByKeyAndWindow(Reducer reducer, Window window) { - return groupByKey().window(window).reduceByKey(reducer); + return partitionByKey().window(window).reduceByKey(reducer); } /** @@ -137,10 +170,17 @@ public PairStream peek(Consumer> action) { return toPairStream(super.peek(action)); } + /** + * {@inheritDoc} + */ + public PairStream filter(Predicate> predicate) { + return toPairStream(super.filter(predicate)); + } + /** * Join the values of this stream with the values having the same key from the other stream. *

- * Note: The parallelism and windowing parameters (if windowed) of this stream is carried forward to the joined stream. + * Note: The parallelism of this stream is carried forward to the joined stream. *

* * @param otherStream the other stream @@ -151,10 +191,52 @@ public PairStream> join(PairStream otherStream) { return join(otherStream, new PairValueJoiner<>()); } + /** + * Does a left outer join of the values of this stream with the values having the same key from the other stream. + *

+ * Note: The parallelism of this stream is carried forward to the joined stream. + *

+ * + * @param otherStream the other stream + * @param the type of the values in the other stream + * @return the new stream + */ + public PairStream> leftOuterJoin(PairStream otherStream) { + return leftOuterJoin(otherStream, new PairValueJoiner<>()); + } + + /** + * Does a right outer join of the values of this stream with the values having the same key from the other stream. + *

+ * Note: The parallelism of this stream is carried forward to the joined stream. + *

+ * + * @param otherStream the other stream + * @param the type of the values in the other stream + * @return the new stream + */ + public PairStream> rightOuterJoin(PairStream otherStream) { + return rightOuterJoin(otherStream, new PairValueJoiner<>()); + } + + /** + * Does a full outer join of the values of this stream with the values having the same key from the other stream. + *

+ * Note: The parallelism of this stream is carried forward to the joined stream. + *

+ * + * @param otherStream the other stream + * @param the type of the values in the other stream + * @return the new stream + */ + public PairStream> fullOuterJoin(PairStream otherStream) { + return fullOuterJoin(otherStream, new PairValueJoiner<>()); + } + /** * Join the values of this stream with the values having the same key from the other stream. *

- * Note: The parallelism and windowing parameters (if windowed) of this stream is carried forward to the joined stream. + * Note: The parallelism of this stream is carried forward to the joined stream. *

* * @param otherStream the other stream @@ -165,11 +247,78 @@ public PairStream> join(PairStream otherStream) { */ public PairStream join(PairStream otherStream, ValueJoiner valueJoiner) { - String leftStream = stream; - String rightStream = otherStream.stream; - Node joinNode = addProcessorNode(new JoinProcessor<>(leftStream, rightStream, valueJoiner), KEY_VALUE); - addNode(otherStream.getNode(), joinNode, joinNode.getParallelism()); - return new PairStream<>(streamBuilder, joinNode); + return partitionByKey() + .joinPartition( + otherStream.partitionByKey(), + valueJoiner, + JoinProcessor.JoinType.INNER, + JoinProcessor.JoinType.INNER); + } + + /** + * Does a left outer join of the values of this stream with the values having the same key from the other stream. + *

+ * Note: The parallelism of this stream is carried forward to the joined stream. + *

+ * + * @param otherStream the other stream + * @param valueJoiner the {@link ValueJoiner} + * @param the type of the values resulting from the join + * @param the type of the values in the other stream + * @return the new stream + */ + public PairStream leftOuterJoin(PairStream otherStream, + ValueJoiner valueJoiner) { + return partitionByKey() + .joinPartition( + otherStream.partitionByKey(), + valueJoiner, + JoinProcessor.JoinType.OUTER, + JoinProcessor.JoinType.INNER); + } + + /** + * Does a right outer join of the values of this stream with the values having the same key from the other stream. + *

+ * Note: The parallelism of this stream is carried forward to the joined stream. + *

+ * + * @param otherStream the other stream + * @param valueJoiner the {@link ValueJoiner} + * @param the type of the values resulting from the join + * @param the type of the values in the other stream + * @return the new stream + */ + public PairStream rightOuterJoin(PairStream otherStream, + ValueJoiner valueJoiner) { + return partitionByKey() + .joinPartition( + otherStream.partitionByKey(), + valueJoiner, + JoinProcessor.JoinType.INNER, + JoinProcessor.JoinType.OUTER); + } + + /** + * Does a full outer join of the values of this stream with the values having the same key from the other stream. + *

+ * Note: The parallelism of this stream is carried forward to the joined stream. + *

+ * + * @param otherStream the other stream + * @param valueJoiner the {@link ValueJoiner} + * @param the type of the values resulting from the join + * @param the type of the values in the other stream + * @return the new stream + */ + public PairStream fullOuterJoin(PairStream otherStream, + ValueJoiner valueJoiner) { + return partitionByKey() + .joinPartition( + otherStream.partitionByKey(), + valueJoiner, + JoinProcessor.JoinType.OUTER, + JoinProcessor.JoinType.OUTER); } /** @@ -193,7 +342,7 @@ public PairStream repartition(int parallelism) { */ @Override @SuppressWarnings("unchecked") - public PairStream[] branch(Predicate>... predicates) { + public PairStream[] branch(Predicate>... predicates) { List> pairStreams = new ArrayList<>(); for (Stream> stream : super.branch(predicates)) { pairStreams.add(toPairStream(stream)); @@ -202,40 +351,178 @@ public PairStream[] branch(Predicate>... predicates) { } /** - * Update the state by applying the given aggregator to the previous state of the + * Update the state by applying the given state update function to the previous state of the * key and the new value for the key. This internally uses {@link org.apache.storm.topology.IStatefulBolt} * to save the state. Use {@link Config#TOPOLOGY_STATE_PROVIDER} to choose the state implementation. * - * @param aggregator the aggregator + * @param stateUpdateFn the state update function * @param the result type * @return the {@link StreamState} which can be used to query the state */ - public StreamState updateStateByKey(Aggregator aggregator) { + public StreamState updateStateByKey(R initialValue, + BiFunction stateUpdateFn) { + return updateStateByKey(StateUpdater.of(initialValue, stateUpdateFn)); + } + + /** + * Update the state by applying the given state update function to the previous state of the + * key and the new value for the key. This internally uses {@link org.apache.storm.topology.IStatefulBolt} + * to save the state. Use {@link Config#TOPOLOGY_STATE_PROVIDER} to choose the state implementation. + * + * @param stateUpdater the state updater + * @param the result type + * @return the {@link StreamState} which can be used to query the state + */ + public StreamState updateStateByKey(StateUpdater stateUpdater) { + return partitionByKey().updateStateByKeyPartition(stateUpdater); + } + + private StreamState updateStateByKeyPartition(StateUpdater stateUpdater) { return new StreamState<>( - new PairStream<>(streamBuilder, addProcessorNode(new UpdateStateByKeyProcessor<>(aggregator), KEY_VALUE))); + new PairStream<>(streamBuilder, + addProcessorNode(new UpdateStateByKeyProcessor<>(stateUpdater), KEY_VALUE, true))); + } + + private PairStream joinPartition(PairStream otherStream, + ValueJoiner valueJoiner, + JoinProcessor.JoinType leftType, + JoinProcessor.JoinType rightType) { + String leftStream = stream; + String rightStream = otherStream.stream; + Node joinNode = addProcessorNode( + new JoinProcessor<>(leftStream, rightStream, valueJoiner, leftType, rightType), + KEY_VALUE, + true); + addNode(otherStream.getNode(), joinNode, joinNode.getParallelism()); + return new PairStream<>(streamBuilder, joinNode); + } + + private PairStream partitionByKey() { + return shouldPartitionByKey() ? partitionBy(KEY) : this; + } + + private boolean shouldPartitionByKey() { + if (node.getParallelism() == 1) { + return false; + } + /* + * if the current processor preserves the key and is + * already partitioned on key, skip the re-partition. + */ + if (node instanceof ProcessorNode) { + ProcessorNode pn = (ProcessorNode) node; + Fields fields = pn.getGroupingInfo() == null ? null : pn.getGroupingInfo().getFields(); + if (pn.isPreservesKey() && fields != null && fields.equals(KEY)) { + return false; + } + } + return true; } private PairStream partitionBy(Fields fields) { + return partitionBy(fields, node.parallelism); + } + + private PairStream partitionBy(Fields fields, int parallelism) { return new PairStream<>( streamBuilder, - addNode(new PartitionNode(stream, node.getOutputFields(), GroupingInfo.fields(fields)))); + addNode(node, new PartitionNode(stream, node.getOutputFields(), GroupingInfo.fields(fields)), parallelism)); } private PairStream toPairStream(Stream> stream) { return new PairStream<>(stream.streamBuilder, stream.node); } + private PairStream aggregatePartition(CombinerAggregator aggregator) { + return new PairStream<>(streamBuilder, + addProcessorNode(new AggregateByKeyProcessor<>(aggregator), KEY_VALUE, true)); + } + + private
PairStream combinePartition(CombinerAggregator aggregator) { + return new PairStream<>(streamBuilder, + addProcessorNode(new AggregateByKeyProcessor<>(aggregator, true), KEY_VALUE, true)); + } + + private PairStream merge(CombinerAggregator aggregator) { + return new PairStream<>(streamBuilder, + addProcessorNode(new MergeAggregateByKeyProcessor<>(aggregator), KEY_VALUE, true)); + } + + private PairStream reducePartition(Reducer reducer) { + return new PairStream<>(streamBuilder, + addProcessorNode(new ReduceByKeyProcessor<>(reducer), KEY_VALUE, true)); + } + + // if re-partitioning is involved, does a per-partition aggregate by key before emitting the results downstream + private PairStream combineByKey(CombinerAggregator aggregator) { + if (shouldPartitionByKey()) { + if (node instanceof ProcessorNode) { + if (node.isWindowed()) { + return combinePartition(aggregator).partitionBy(KEY).merge(aggregator); + } + } else if (node instanceof WindowNode) { + Set parents = node.getParents(); + Optional nonWindowed = parents.stream().filter(p -> !p.isWindowed()).findAny(); + if (!nonWindowed.isPresent()) { + parents.forEach(p -> { + Node localAggregateNode = makeProcessorNode( + new AggregateByKeyProcessor<>(aggregator, true), KEY_VALUE, true); + streamBuilder.insert(p, localAggregateNode); + }); + return ((PairStream)partitionBy(KEY)).merge(aggregator); + } + } + return partitionBy(KEY).aggregatePartition(aggregator); + } else { + return aggregatePartition(aggregator); + } + } + + // if re-partitioning is involved, does a per-partition reduce by key before emitting the results downstream + private PairStream combineByKey(Reducer reducer) { + if (shouldPartitionByKey()) { + if (node instanceof ProcessorNode) { + if (node.isWindowed()) { + return reducePartition(reducer).partitionBy(KEY).reducePartition(reducer); + } + } else if (node instanceof WindowNode) { + for (Node p : node.getParents()) { + if (p.isWindowed()) { + Node localReduceNode = makeProcessorNode(new ReduceByKeyProcessor<>(reducer), KEY_VALUE, true); + streamBuilder.insert(p, localReduceNode); + } + } + } + return partitionBy(KEY).reducePartition(reducer); + } else { + return reducePartition(reducer); + } + } + // used internally to merge values in groupByKeyAndWindow - private static class MergeValues implements Aggregator> { + private static class MergeValues implements CombinerAggregator, ArrayList> { @Override public ArrayList init() { return new ArrayList<>(); } @Override - public ArrayList apply(V value, ArrayList aggregate) { + public ArrayList apply(ArrayList aggregate, V value) { aggregate.add(value); return aggregate; } + + @Override + public ArrayList merge(ArrayList accum1, ArrayList accum2) { + ArrayList res = new ArrayList(); + res.addAll(accum1); + res.addAll(accum2); + return res; + } + + @Override + public ArrayList result(ArrayList accum) { + return accum; + } } } diff --git a/storm-core/src/jvm/org/apache/storm/streams/PartitionNode.java b/storm-core/src/jvm/org/apache/storm/streams/PartitionNode.java index ca92deff2a9..90b4d7cf590 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/PartitionNode.java +++ b/storm-core/src/jvm/org/apache/storm/streams/PartitionNode.java @@ -25,18 +25,13 @@ * aggregate/reduce (global grouping), state query (all grouping). */ class PartitionNode extends Node { - private final GroupingInfo groupingInfo; PartitionNode(String outputStream, Fields outputFields, GroupingInfo groupingInfo) { - super(outputStream, outputFields); - this.groupingInfo = groupingInfo; + super(outputStream, outputFields, groupingInfo); } PartitionNode(String outputStream, Fields outputFields) { this(outputStream, outputFields, GroupingInfo.shuffle()); } - GroupingInfo getGroupingInfo() { - return groupingInfo; - } } diff --git a/storm-core/src/jvm/org/apache/storm/streams/ProcessorBoltDelegate.java b/storm-core/src/jvm/org/apache/storm/streams/ProcessorBoltDelegate.java index 5bc6fff1a5f..3f3b5c1da01 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/ProcessorBoltDelegate.java +++ b/storm-core/src/jvm/org/apache/storm/streams/ProcessorBoltDelegate.java @@ -18,7 +18,10 @@ package org.apache.storm.streams; import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.HashBasedTable; import com.google.common.collect.Multimap; +import com.google.common.collect.Table; +import org.apache.storm.generated.GlobalStreamId; import org.apache.storm.streams.processors.ChainedProcessorContext; import org.apache.storm.streams.processors.EmittingProcessorContext; import org.apache.storm.streams.processors.ForwardingProcessorContext; @@ -55,8 +58,9 @@ class ProcessorBoltDelegate implements Serializable { private OutputCollector outputCollector; private final List outgoingProcessors = new ArrayList<>(); private final Set emittingProcessorContexts = new HashSet<>(); - private final Map> punctuationState = new HashMap<>(); + private final Table punctuationState = HashBasedTable.create(); private Multimap streamToInitialProcessors; + private final Map streamToInputTaskCount = new HashMap<>(); private String timestampField; ProcessorBoltDelegate(String id, DirectedGraph graph, List nodes) { @@ -114,6 +118,9 @@ void prepare(Map stormConf, TopologyContext context, OutputCollector collector) ctx.setTimestampField(timestampField); } } + for (String stream : streamToInitialProcessors.keySet()) { + streamToInputTaskCount.put(stream, getStreamInputTaskCount(context, stream)); + } } void declareOutputFields(OutputFieldsDeclarer declarer) { @@ -127,6 +134,12 @@ void declareOutputFields(OutputFieldsDeclarer declarer) { fields.add(timestampField); declarer.declareStream(stream, new Fields(fields)); } + /* + * Declare a separate 'punctuation' stream per output stream so that the receiving bolt + * can subscribe to this stream with 'ALL' grouping and process the punctuation once it + * receives from all upstream tasks. + */ + declarer.declareStream(StreamUtil.getPunctuationStream(stream), StreamUtil.getPunctuationFields()); } } } @@ -168,20 +181,32 @@ void processAndAck(Tuple input) { void process(Object value, String sourceStreamId) { LOG.debug("Process value {}, sourceStreamId {}", value, sourceStreamId); + if (StreamUtil.isPunctuation(value)) { + punctuateInitialProcessors(sourceStreamId); + } else { + executeInitialProcessors(value, sourceStreamId); + } + } + + private void punctuateInitialProcessors(String punctuationStreamId) { + String sourceStreamId = StreamUtil.getSourceStream(punctuationStreamId); Collection initialProcessors = streamToInitialProcessors.get(sourceStreamId); for (ProcessorNode processorNode : initialProcessors) { - Processor processor = processorNode.getProcessor(); - if (StreamUtil.isPunctuation(value)) { - if (shouldPunctuate(processorNode, sourceStreamId)) { - processor.punctuate(null); - clearPunctuationState(processorNode); - } - } else { - processor.execute(value, sourceStreamId); + if (shouldPunctuate(processorNode, sourceStreamId)) { + processorNode.getProcessor().punctuate(null); + clearPunctuationState(processorNode); } } } + private void executeInitialProcessors(Object value, String sourceStreamId) { + Collection initialProcessors = streamToInitialProcessors.get(sourceStreamId); + for (ProcessorNode processorNode : initialProcessors) { + Processor processor = processorNode.getProcessor(); + processor.execute(value, sourceStreamId); + } + } + void setStreamToInitialProcessors(Multimap streamToInitialProcessors) { this.streamToInitialProcessors = streamToInitialProcessors; } @@ -225,9 +250,6 @@ private ProcessorContext createEmittingContext(ProcessorNode processorNode) { List emittingContexts = new ArrayList<>(); for (String stream : processorNode.getOutputStreams()) { EmittingProcessorContext emittingContext = new EmittingProcessorContext(processorNode, outputCollector, stream); - if (StreamUtil.isSinkStream(stream)) { - emittingContext.setEmitPunctuation(false); - } emittingContexts.add(emittingContext); } emittingProcessorContexts.addAll(emittingContexts); @@ -257,24 +279,48 @@ private Set getChildNodes(Node node) { return children; } - // if we received punctuation from all parent windowed streams + // for the given processor node, if we received punctuation from all tasks of its parent windowed streams private boolean shouldPunctuate(ProcessorNode processorNode, String sourceStreamId) { - if (processorNode.getWindowedParentStreams().size() <= 1) { - return true; + if (!processorNode.getWindowedParentStreams().isEmpty()) { + updateCount(processorNode, sourceStreamId); + if (punctuationState.row(processorNode).size() != processorNode.getWindowedParentStreams().size()) { + return false; + } + // size matches, check if the streams are expected + Set receivedStreams = punctuationState.row(processorNode).keySet(); + if (!receivedStreams.equals(processorNode.getWindowedParentStreams())) { + throw new IllegalStateException("Received punctuation from streams " + receivedStreams + " expected " + + processorNode.getWindowedParentStreams()); + } + for (String receivedStream : receivedStreams) { + Integer expected = streamToInputTaskCount.get(receivedStream); + if (expected == null) { + throw new IllegalStateException("Punctuation received on unexpected stream '" + receivedStream + + "' for which input task count is not set."); + } + if (punctuationState.get(processorNode, receivedStream) < streamToInputTaskCount.get(receivedStream)) { + return false; + } + } } - Set receivedStreams = punctuationState.get(processorNode); - if (receivedStreams == null) { - receivedStreams = new HashSet<>(); - punctuationState.put(processorNode, receivedStreams); + return true; + } + + private void updateCount(ProcessorNode processorNode, String sourceStreamId) { + Integer count = punctuationState.get(processorNode, sourceStreamId); + if (count == null) { + punctuationState.put(processorNode, sourceStreamId, 1); + } else { + punctuationState.put(processorNode, sourceStreamId, count + 1); } - receivedStreams.add(sourceStreamId); - return receivedStreams.equals(processorNode.getWindowedParentStreams()); } private void clearPunctuationState(ProcessorNode processorNode) { - Set state = punctuationState.get(processorNode); - if (state != null) { - state.clear(); + if (!punctuationState.isEmpty()) { + Map state = punctuationState.row(processorNode); + if (!state.isEmpty()) { + state.clear(); + } } } @@ -282,4 +328,21 @@ private boolean isPair(Tuple input) { return input.size() == (timestampField == null ? 2 : 3); } + private int getStreamInputTaskCount(TopologyContext context, String stream) { + int count = 0; + for (GlobalStreamId inputStream : context.getThisSources().keySet()) { + if (stream.equals(getStreamId(inputStream))) { + count += context.getComponentTasks(inputStream.get_componentId()).size(); + } + } + return count; + } + + private String getStreamId(GlobalStreamId globalStreamId) { + if (globalStreamId.get_componentId().startsWith("spout")) { + return globalStreamId.get_componentId() + globalStreamId.get_streamId(); + } + return globalStreamId.get_streamId(); + } + } diff --git a/storm-core/src/jvm/org/apache/storm/streams/ProcessorNode.java b/storm-core/src/jvm/org/apache/storm/streams/ProcessorNode.java index 4771f4f8b3f..b68dd48264f 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/ProcessorNode.java +++ b/storm-core/src/jvm/org/apache/storm/streams/ProcessorNode.java @@ -32,32 +32,29 @@ public class ProcessorNode extends Node { private final Processor processor; private final boolean isBatch; - private boolean windowed; + private final boolean preservesKey; // Windowed parent streams private Set windowedParentStreams = Collections.emptySet(); - public ProcessorNode(Processor processor, String outputStream, Fields outputFields) { + public ProcessorNode(Processor processor, String outputStream, Fields outputFields, boolean preservesKey) { super(outputStream, outputFields); this.isBatch = processor instanceof BatchProcessor; this.processor = processor; + this.preservesKey = preservesKey; } - public Processor getProcessor() { - return processor; + public ProcessorNode(Processor processor, String outputStream, Fields outputFields) { + this(processor, outputStream, outputFields, false); } - public boolean isWindowed() { - return windowed; + public Processor getProcessor() { + return processor; } public boolean isBatch() { return isBatch; } - public void setWindowed(boolean windowed) { - this.windowed = windowed; - } - public Set getWindowedParentStreams() { return Collections.unmodifiableSet(windowedParentStreams); } @@ -70,11 +67,16 @@ void setWindowedParentStreams(Set windowedParentStreams) { this.windowedParentStreams = new HashSet<>(windowedParentStreams); } + public boolean isPreservesKey() { + return preservesKey; + } + @Override public String toString() { return "ProcessorNode{" + "processor=" + processor + - ", windowed=" + windowed + + ", isBatch=" + isBatch + + ", preservesKey=" + preservesKey + ", windowedParentStreams=" + windowedParentStreams + "} " + super.toString(); } diff --git a/storm-core/src/jvm/org/apache/storm/streams/Stream.java b/storm-core/src/jvm/org/apache/storm/streams/Stream.java index e50e7a25d5d..d553390a987 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/Stream.java +++ b/storm-core/src/jvm/org/apache/storm/streams/Stream.java @@ -17,7 +17,8 @@ */ package org.apache.storm.streams; -import org.apache.storm.streams.operations.Aggregator; +import org.apache.storm.streams.operations.BiFunction; +import org.apache.storm.streams.operations.CombinerAggregator; import org.apache.storm.streams.operations.Consumer; import org.apache.storm.streams.operations.FlatMapFunction; import org.apache.storm.streams.operations.Function; @@ -27,12 +28,14 @@ import org.apache.storm.streams.operations.Predicate; import org.apache.storm.streams.operations.PrintConsumer; import org.apache.storm.streams.operations.Reducer; +import org.apache.storm.streams.operations.aggregators.Count; import org.apache.storm.streams.processors.AggregateProcessor; import org.apache.storm.streams.processors.BranchProcessor; import org.apache.storm.streams.processors.FilterProcessor; import org.apache.storm.streams.processors.FlatMapProcessor; import org.apache.storm.streams.processors.ForEachProcessor; import org.apache.storm.streams.processors.MapProcessor; +import org.apache.storm.streams.processors.MergeAggregateProcessor; import org.apache.storm.streams.processors.PeekProcessor; import org.apache.storm.streams.processors.Processor; import org.apache.storm.streams.processors.ReduceProcessor; @@ -42,9 +45,13 @@ import org.apache.storm.topology.IRichBolt; import org.apache.storm.tuple.Fields; import org.apache.storm.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.List; +import java.util.Optional; +import java.util.Set; /** * Represents a stream of values. @@ -52,6 +59,8 @@ * @param the type of the value */ public class Stream { + private static final Logger LOG = LoggerFactory.getLogger(Stream.class); + protected static final Fields KEY = new Fields("key"); protected static final Fields VALUE = new Fields("value"); protected static final Fields KEY_VALUE = new Fields("key", "value"); @@ -79,7 +88,7 @@ private Stream(StreamBuilder streamBuilder, Node node, String stream) { * @return the new stream */ public Stream filter(Predicate predicate) { - return new Stream<>(streamBuilder, addProcessorNode(new FilterProcessor<>(predicate), VALUE)); + return new Stream<>(streamBuilder, addProcessorNode(new FilterProcessor<>(predicate), VALUE, true)); } /** @@ -100,7 +109,7 @@ public Stream map(Function function) { * @param the value type * @return the new stream of key-value pairs */ - public PairStream mapToPair(PairFunction function) { + public PairStream mapToPair(PairFunction function) { return new PairStream<>(streamBuilder, addProcessorNode(new MapProcessor<>(function), KEY_VALUE)); } @@ -113,7 +122,7 @@ public PairStream mapToPair(PairFunction function) { * @param function a mapping function to be applied to each value in this stream which produces new values. * @return the new stream */ - public Stream flatMap(FlatMapFunction function) { + public Stream flatMap(FlatMapFunction function) { return new Stream<>(streamBuilder, addProcessorNode(new FlatMapProcessor<>(function), VALUE)); } @@ -128,7 +137,7 @@ public Stream flatMap(FlatMapFunction function) { * @see #flatMap(FlatMapFunction) * @see #mapToPair(PairFunction) */ - public PairStream flatMapToPair(PairFlatMapFunction function) { + public PairStream flatMapToPair(PairFlatMapFunction function) { return new PairStream<>(streamBuilder, addProcessorNode(new FlatMapProcessor<>(function), KEY_VALUE)); } @@ -173,12 +182,11 @@ public void forEach(Consumer action) { * @return the new stream */ public Stream peek(Consumer action) { - return new Stream<>(streamBuilder, addProcessorNode(new PeekProcessor<>(action), node.getOutputFields())); + return new Stream<>(streamBuilder, addProcessorNode(new PeekProcessor<>(action), node.getOutputFields(), true)); } /** - * Aggregates the values in this stream using the aggregator. This does a global aggregation, i.e. the elements - * across all the partitions are forwarded to a single task for computing the aggregate. + * Aggregates the values in this stream using the aggregator. This does a global aggregation of values across all partitions. *

* If the stream is windowed, the aggregate result is emitted after each window activation and represents the * aggregate of elements that fall within that window. @@ -186,15 +194,52 @@ public Stream peek(Consumer action) { *

* * @param aggregator the aggregator + * @param
the accumulator type * @param the result type * @return the new stream */ - public Stream aggregate(Aggregator aggregator) { - return new Stream<>(streamBuilder, global().addProcessorNode(new AggregateProcessor<>(aggregator), VALUE)); + public Stream aggregate(CombinerAggregator aggregator) { + return combine(aggregator); + } + + /** + * Aggregates the values in this stream using the given initial value, accumulator and combiner. This does a global + * aggregation of values across all partitions. + *

+ * If the stream is windowed, the aggregate result is emitted after each window activation and represents the + * aggregate of elements that fall within that window. + * If the stream is not windowed, the aggregate result is emitted as each new element in the stream is processed. + *

+ * + * @param initialValue the initial value of the result + * @param accumulator the accumulator + * @param combiner the combiner + * @param the result type + * @return the new stream + */ + public Stream aggregate(R initialValue, + BiFunction accumulator, + BiFunction combiner) { + return combine(CombinerAggregator.of(initialValue, accumulator, combiner)); + } + + /** + * Counts the number of values in this stream. This does a global count of values across all partitions. + *

+ * If the stream is windowed, the counts are emitted after each window activation and represents the + * count of elements that fall within that window. + * If the stream is not windowed, the count is emitted as each new element in the stream is processed. + *

+ * + * @return the new stream + */ + public Stream count() { + return aggregate(new Count<>()); } /** * Performs a reduction on the elements of this stream, by repeatedly applying the reducer. + * This does a global reduction of values across all partitions. *

* If the stream is windowed, the result is emitted after each window activation and represents the * reduction of elements that fall within that window. @@ -205,7 +250,7 @@ public Stream aggregate(Aggregator aggregator) { * @return the new stream */ public Stream reduce(Reducer reducer) { - return new Stream<>(streamBuilder, global().addProcessorNode(new ReduceProcessor<>(reducer), VALUE)); + return combine(reducer); } /** @@ -219,6 +264,10 @@ public Stream repartition(int parallelism) { if (parallelism < 1) { throw new IllegalArgumentException("Parallelism should be >= 1"); } + if (node.getParallelism() == parallelism) { + LOG.debug("Node's current parallelism {}, new parallelism {}", node.getParallelism(), parallelism); + return this; + } Node partitionNode = addNode(node, new PartitionNode(stream, node.getOutputFields()), parallelism); return new Stream<>(streamBuilder, partitionNode); } @@ -233,12 +282,12 @@ public Stream repartition(int parallelism) { * @return an array of result streams (branches) corresponding to the given predicates */ @SuppressWarnings("unchecked") - public Stream[] branch(Predicate... predicates) { + public Stream[] branch(Predicate... predicates) { List> childStreams = new ArrayList<>(); if (predicates.length > 0) { BranchProcessor branchProcessor = new BranchProcessor<>(); Node branchNode = addProcessorNode(branchProcessor, VALUE); - for (Predicate predicate : predicates) { + for (Predicate predicate : predicates) { // create a child node (identity) per branch ProcessorNode child = makeProcessorNode(new MapProcessor<>(new IdentityFunction<>()), node.getOutputFields()); String branchStream = child.getOutputStreams().iterator().next() + "-branch"; @@ -321,10 +370,8 @@ public void to(IBasicBolt bolt, int parallelism) { * @return the result stream */ public PairStream stateQuery(StreamState streamState) { - // need all grouping for state query since the state is local + // need all grouping for state query since the state is per-task Node node = all().addProcessorNode(new StateQueryProcessor<>(streamState), KEY_VALUE); - // add 'updateState' node as parent so that state query gets processed after update state - addNode(streamState.getUpdateStateNode(), node, node.getParallelism()); return new PairStream<>(streamBuilder, node); } @@ -337,13 +384,17 @@ Node addNode(Node parent, Node child, int parallelism) { } Node addNode(Node child) { - return addNode(this.node, child); + return addNode(node, child); } Node addProcessorNode(Processor processor, Fields outputFields) { return addNode(makeProcessorNode(processor, outputFields)); } + Node addProcessorNode(Processor processor, Fields outputFields, boolean preservesKey) { + return addNode(makeProcessorNode(processor, outputFields, preservesKey)); + } + String getStream() { return stream; } @@ -356,12 +407,16 @@ private Node addNode(Node parent, Node child, String parentStreamId) { return streamBuilder.addNode(parent, child, parentStreamId); } - private Node addNode(Node child, int parallelism, String parentStreamId) { - return streamBuilder.addNode(this.node, child, parallelism, parentStreamId); + private Node addNode(Node parent, Node child, String parentStreamId, int parallelism) { + return streamBuilder.addNode(parent, child, parentStreamId, parallelism); } private ProcessorNode makeProcessorNode(Processor processor, Fields outputFields) { - return new ProcessorNode(processor, UniqueIdGen.getInstance().getUniqueStreamId(), outputFields); + return makeProcessorNode(processor, outputFields, false); + } + + ProcessorNode makeProcessorNode(Processor processor, Fields outputFields, boolean preservesKey) { + return new ProcessorNode(processor, UniqueIdGen.getInstance().getUniqueStreamId(), outputFields, preservesKey); } private void addSinkNode(SinkNode sinkNode, int parallelism) { @@ -369,15 +424,9 @@ private void addSinkNode(SinkNode sinkNode, int parallelism) { sinkNode.setComponentId(boltId); sinkNode.setParallelism(parallelism); if (node instanceof SpoutNode) { - addNode(sinkNode, parallelism, Utils.DEFAULT_STREAM_ID); + addNode(node, sinkNode, Utils.DEFAULT_STREAM_ID, parallelism); } else { - /* - * add a stream__sink stream to the current node (parent) for emitting - * just the values (no punctuation) to the bolt. - */ - String sinkStream = StreamUtil.getSinkStream(stream); - node.addOutputStream(sinkStream); - addNode(sinkNode, parallelism, sinkStream); + addNode(node, sinkNode, parallelism); } } @@ -387,7 +436,79 @@ private Stream global() { } private Stream all() { + if (node.getParallelism() == 1) { + return this; + } Node partitionNode = addNode(new PartitionNode(stream, node.getOutputFields(), GroupingInfo.all())); return new Stream<>(streamBuilder, partitionNode); } + + private boolean shouldPartition() { + return node.getParallelism() > 1; + } + + private Stream combinePartition(CombinerAggregator aggregator) { + return new Stream<>(streamBuilder, + addProcessorNode(new AggregateProcessor<>(aggregator, true), VALUE, true)); + } + + private Stream merge(CombinerAggregator aggregator) { + return new Stream<>(streamBuilder, + addProcessorNode(new MergeAggregateProcessor<>(aggregator), VALUE)); + } + + private Stream aggregatePartition(CombinerAggregator aggregator) { + return new Stream<>(streamBuilder, addProcessorNode(new AggregateProcessor<>(aggregator), VALUE)); + } + + private Stream reducePartition(Reducer reducer) { + return new Stream<>(streamBuilder, addProcessorNode(new ReduceProcessor<>(reducer), VALUE)); + } + + // if re-partitioning is involved, does a per-partition aggregate before emitting the results downstream + private Stream combine(CombinerAggregator aggregator) { + if (shouldPartition()) { + if (node instanceof ProcessorNode) { + if (node.isWindowed()) { + return combinePartition(aggregator).global().merge(aggregator); + } + } else if (node instanceof WindowNode) { + Set parents = node.getParents(); + Optional nonWindowed = parents.stream().filter(p -> !p.isWindowed()).findAny(); + if (!nonWindowed.isPresent()) { + parents.forEach(p -> { + Node localAggregateNode = makeProcessorNode( + new AggregateProcessor<>(aggregator, true), VALUE, true); + streamBuilder.insert(p, localAggregateNode); + }); + return ((Stream)global()).merge(aggregator); + } + } + return global().aggregatePartition(aggregator); + } else { + return aggregatePartition(aggregator); + } + } + + // if re-partitioning is involved, does a per-partition reduce before emitting the results downstream + private Stream combine(Reducer reducer) { + if (shouldPartition()) { + if (node instanceof ProcessorNode) { + if (node.isWindowed()) { + return reducePartition(reducer).global().reducePartition(reducer); + } + } else if (node instanceof WindowNode) { + for (Node p : node.getParents()) { + if (p.isWindowed()) { + Node localReduceNode = makeProcessorNode(new ReduceProcessor<>(reducer), VALUE); + streamBuilder.insert(p, localReduceNode); + } + } + } + return global().reducePartition(reducer); + } else { + return reducePartition(reducer); + } + } + } diff --git a/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java b/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java index e19a0c63bda..7dff25d1afc 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java +++ b/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java @@ -30,8 +30,10 @@ import org.apache.storm.streams.operations.mappers.TupleValueMapper; import org.apache.storm.streams.processors.JoinProcessor; import org.apache.storm.streams.processors.MapProcessor; +import org.apache.storm.streams.processors.Processor; import org.apache.storm.streams.processors.StateQueryProcessor; import org.apache.storm.streams.processors.StatefulProcessor; +import org.apache.storm.streams.processors.UpdateStateByKeyProcessor; import org.apache.storm.streams.windowing.Window; import org.apache.storm.topology.BoltDeclarer; import org.apache.storm.topology.IBasicBolt; @@ -195,18 +197,30 @@ public StormTopology build() { } Node addNode(Node parent, Node child) { - return addNode(parent, child, parent.getParallelism(), parent.getOutputStreams().iterator().next()); + return addNode(parent, child, parent.getOutputStreams().iterator().next(), parent.getParallelism()); } Node addNode(Node parent, Node child, int parallelism) { - return addNode(parent, child, parallelism, parent.getOutputStreams().iterator().next()); + return addNode(parent, child, parent.getOutputStreams().iterator().next(), parallelism); + } + + // insert child in-between parent and its current child nodes + Node insert(Node parent, Node child) { + Node newChild = addNode(parent, child); + for (Edge edge : graph.outgoingEdgesOf(parent)) { + Node oldChild = edge.getTarget(); + graph.removeEdge(parent, oldChild); + oldChild.removeParentStreams(parent); + addNode(newChild, oldChild); + } + return newChild; } Node addNode(Node parent, Node child, String parentStreamId) { - return addNode(parent, child, parent.getParallelism(), parentStreamId); + return addNode(parent, child, parentStreamId, parent.getParallelism()); } - Node addNode(Node parent, Node child, int parallelism, String parentStreamId) { + Node addNode(Node parent, Node child, String parentStreamId, int parallelism) { graph.addVertex(child); graph.addEdge(parent, child); child.setParallelism(parallelism); @@ -215,34 +229,62 @@ Node addNode(Node parent, Node child, int parallelism, String parentStreamId) { } else { child.addParentStream(parent, parentStreamId); } + if (!(child instanceof PartitionNode)) { + if (child.getGroupingInfo() != null) { + if (!child.getGroupingInfo().equals(parent.getGroupingInfo())) { + throw new IllegalStateException("Trying to assign grouping info for node" + + " with current grouping info: " + child.getGroupingInfo() + + " to: " + parent.getGroupingInfo() + " Node: " + child); + } + } else { + child.setGroupingInfo(parent.getGroupingInfo()); + } + } + if (!(child instanceof WindowNode) && !child.isWindowed()) { + child.setWindowed(parent.isWindowed()); + } return child; } private PriorityQueue queue() { // min-heap return new PriorityQueue<>(new Comparator() { + /* + * Nodes in the descending order of priority. + * ProcessorNode has higher priority than partition and window nodes + * so that the topological order iterator will group as many processor nodes together as possible. + * UpdateStateByKeyProcessor has a higher priority than StateQueryProcessor so that StateQueryProcessor + * can be mapped to the same StatefulBolt that UpdateStateByKeyProcessor is part of. + */ + Map, Integer> p = new HashMap<>(); + { + p.put(SpoutNode.class, 0); + p.put(UpdateStateByKeyProcessor.class, 1); + p.put(ProcessorNode.class, 2); + p.put(StateQueryProcessor.class, 3); + p.put(PartitionNode.class, 4); + p.put(WindowNode.class, 5); + p.put(SinkNode.class, 6); + } @Override public int compare(Node n1, Node n2) { - return getPriority(n1.getClass()) - getPriority(n2.getClass()); + return getPriority(n1) - getPriority(n2); } - private int getPriority(Class clazz) { - /* - * Nodes in the descending order of priority. - * ProcessorNode has the highest priority so that the topological order iterator - * will group as many processor nodes together as possible. - */ - Class[] p = new Class[]{ - ProcessorNode.class, - SpoutNode.class, - SinkNode.class, - PartitionNode.class, - WindowNode.class}; - for (int i = 0; i < p.length; i++) { - if (clazz.equals(p[i])) { - return i; + private int getPriority(Node node) { + Integer priority; + // check if processor has specific priority first + if (node instanceof ProcessorNode) { + Processor processor = ((ProcessorNode) node).getProcessor(); + priority = p.get(processor.getClass()); + if (priority != null) { + return priority; } } + priority = p.get(node.getClass()); + if (priority != null) { + return priority; + } return Integer.MAX_VALUE; } }); @@ -319,7 +361,7 @@ private void updateWindowInfo(WindowNode windowNode) { } } - private Node parentNode(Node curNode) { + Node parentNode(Node curNode) { Set parentNode = parentNodes(curNode); if (parentNode.size() > 1) { throw new IllegalArgumentException("Node " + curNode + " has more than one parent node."); @@ -350,7 +392,6 @@ private void processCurGroup(TopologyBuilder topologyBuilder) { String boltId = UniqueIdGen.getInstance().getUniqueBoltId(); for (ProcessorNode processorNode : curGroup) { processorNode.setComponentId(boltId); - processorNode.setWindowed(isWindowed(processorNode)); processorNode.setWindowedParentStreams(getWindowedParentStreams(processorNode)); } final Set initialProcessors = initialProcessors(curGroup); @@ -397,12 +438,7 @@ public Integer apply(ProcessorNode input) { Set windowNodes = new HashSet<>(); Set parents; for (ProcessorNode processorNode : initialProcessors) { - if (processorNode.getProcessor() instanceof JoinProcessor) { - String leftStream = ((JoinProcessor) processorNode.getProcessor()).getLeftStream(); - parents = processorNode.getParents(leftStream); - } else { - parents = parentNodes(processorNode); - } + parents = parentNodes(processorNode); for (Node node : parents) { if (windowInfo.containsKey(node)) { windowNodes.add(windowInfo.get(node)); @@ -438,7 +474,7 @@ private void addSink(TopologyBuilder topologyBuilder, SinkNode sinkNode) { } for (Node parent : parentNodes(sinkNode)) { for (String stream : sinkNode.getParentStreams(parent)) { - declareStream(boltDeclarer, parent, stream, nodeGroupingInfo.get(parent, stream)); + declareGrouping(boltDeclarer, parent, stream, nodeGroupingInfo.get(parent, stream)); } } } @@ -511,16 +547,8 @@ private StreamBolt addWindowedBolt(TopologyBuilder topologyBuilder, private Set getWindowedParentStreams(ProcessorNode processorNode) { Set res = new HashSet<>(); for (Node parent : parentNodes(processorNode)) { - if (parent instanceof ProcessorNode) { - ProcessorNode pn = (ProcessorNode) parent; - if (pn.isWindowed()) { - res.addAll(Collections2.filter(pn.getOutputStreams(), new Predicate() { - @Override - public boolean apply(String input) { - return !StreamUtil.isSinkStream(input); - } - })); - } + if (parent instanceof ProcessorNode && parent.isWindowed()) { + res.addAll(parent.getOutputStreams()); } } return res; @@ -539,10 +567,14 @@ private Multimap wireBolt(List curGroup, LOG.debug("Parent {} of curNode {} is in curGroup {}", parent, curNode, curGroup); } else { for (String stream : curNode.getParentStreams(parent)) { - declareStream(boltDeclarer, parent, stream, nodeGroupingInfo.get(parent, stream)); + declareGrouping(boltDeclarer, parent, stream, nodeGroupingInfo.get(parent, stream)); // put global stream id for spouts if (parent.getComponentId().startsWith("spout")) { stream = parent.getComponentId() + stream; + } else { + // subscribe to parent's punctuation stream + String punctuationStream = StreamUtil.getPunctuationStream(stream); + declareGrouping(boltDeclarer, parent, punctuationStream, GroupingInfo.all()); } streamToInitialProcessor.put(stream, curNode); } @@ -552,7 +584,7 @@ private Multimap wireBolt(List curGroup, return streamToInitialProcessor; } - private void declareStream(BoltDeclarer boltDeclarer, Node parent, String streamId, GroupingInfo grouping) { + private void declareGrouping(BoltDeclarer boltDeclarer, Node parent, String streamId, GroupingInfo grouping) { if (grouping == null) { boltDeclarer.shuffleGrouping(parent.getComponentId(), streamId); } else { @@ -572,20 +604,4 @@ private Set initialProcessors(List curGroup) { } return nodes; } - - private boolean isWindowed(Node curNode) { - for (Node parent : StreamUtil.getParents(graph, curNode)) { - if (parent instanceof WindowNode) { - return true; - } else if (parent instanceof ProcessorNode) { - ProcessorNode p = (ProcessorNode) parent; - if (p.isWindowed()) { - return true; - } - } else { - return (parent instanceof PartitionNode) && isWindowed(parent); - } - } - return false; - } } diff --git a/storm-core/src/jvm/org/apache/storm/streams/StreamUtil.java b/storm-core/src/jvm/org/apache/storm/streams/StreamUtil.java index 0531ff60bd3..e0bf7d37c3a 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/StreamUtil.java +++ b/storm-core/src/jvm/org/apache/storm/streams/StreamUtil.java @@ -17,11 +17,14 @@ */ package org.apache.storm.streams; +import org.apache.storm.tuple.Fields; import org.jgrapht.DirectedGraph; import java.util.ArrayList; import java.util.List; +import static org.apache.storm.streams.WindowNode.PUNCTUATION; + public class StreamUtil { @SuppressWarnings("unchecked") public static List getParents(DirectedGraph graph, Node node) { @@ -43,20 +46,24 @@ public static List getChildren(DirectedGraph graph, Node node return ret; } - - public static boolean isSinkStream(String streamId) { - return streamId.endsWith("__sink"); + public static boolean isPunctuation(Object value) { + return PUNCTUATION.equals(value); } - public static String getSinkStream(String streamId) { - return streamId + "__sink"; + public static String getPunctuationStream(String stream) { + return stream + PUNCTUATION; } - public static boolean isPunctuation(Object value) { - if (value instanceof Pair) { - value = ((Pair) value).getFirst(); + public static String getSourceStream(String stream) { + int idx = stream.lastIndexOf(PUNCTUATION); + if (idx > 0) { + return stream.substring(0, idx); } - return WindowNode.PUNCTUATION.equals(value); + return stream; + } + + public static Fields getPunctuationFields() { + return new Fields(PUNCTUATION); } } diff --git a/storm-core/src/jvm/org/apache/storm/streams/WindowNode.java b/storm-core/src/jvm/org/apache/storm/streams/WindowNode.java index a0e831a8492..c1783b5e33b 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/WindowNode.java +++ b/storm-core/src/jvm/org/apache/storm/streams/WindowNode.java @@ -29,6 +29,7 @@ public class WindowNode extends Node { WindowNode(Window windowParams, String outputStream, Fields outputFields) { super(outputStream, outputFields); + setWindowed(true); this.windowParams = windowParams; } diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/Aggregator.java b/storm-core/src/jvm/org/apache/storm/streams/operations/BiFunction.java similarity index 62% rename from storm-core/src/jvm/org/apache/storm/streams/operations/Aggregator.java rename to storm-core/src/jvm/org/apache/storm/streams/operations/BiFunction.java index e3feaf4e3a0..7732e470d7b 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/operations/Aggregator.java +++ b/storm-core/src/jvm/org/apache/storm/streams/operations/BiFunction.java @@ -18,25 +18,20 @@ package org.apache.storm.streams.operations; /** - * Interface for aggregating values. + * a function that accepts two arguments and produces a result. * - * @param the original value type - * @param the aggregated value type + * @param the type of the first argument to the function + * @param the type of the second argument to the function + * @param the type of the result of the function */ -public interface Aggregator extends Operation { +public interface BiFunction extends Operation { /** - * The initial value of the aggregate to start with. + * Applies this function to the given arguments. * - * @return the initial value + * @param input1 the first function argument + * @param input2 the second function argument + * @return the function result */ - R init(); - /** - * Returns a new aggregate by applying the value with the current aggregate. - * - * @param value the value to aggregate - * @param aggregate the current aggregate - * @return the new aggregate - */ - R apply(T value, R aggregate); + R apply(T input1, U input2); } diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/CombinerAggregator.java b/storm-core/src/jvm/org/apache/storm/streams/operations/CombinerAggregator.java new file mode 100644 index 00000000000..a74b2c8d411 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/operations/CombinerAggregator.java @@ -0,0 +1,97 @@ +/** + * 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.storm.streams.operations; + +/** + * Interface for aggregating values. + * + * @param the original value type + * @param the accumulator type + * @param the result type + */ +public interface CombinerAggregator extends Operation { + /** + * The initial value of the accumulator to start with. + * + * @return the initial value of the accumulator + */ + A init(); + + /** + * Updates the accumulator by applying the current accumulator with the value. + * + * @param accumulator the current accumulator + * @param value the value + * @return the updated accumulator + */ + A apply(A accumulator, T value); + + /** + * Merges two accumulators and returns the merged accumulator. + * + * @param accum1 the first accumulator + * @param accum2 the second accumulator + * @return the merged accumulator + */ + A merge(A accum1, A accum2); + + /** + * Produces a result value out of the accumulator. + * + * @param accum the accumulator + * @return the result + */ + R result(A accum); + + /** + * A static factory to create a {@link CombinerAggregator} based on initial value, accumulator and combiner. + * + * @param initialValue the initial value of the result to start with + * @param accumulator a function that accumulates values into a partial result + * @param combiner a function that combines partially accumulated results + * @param the value type + * @param the result type + * @return the {@link CombinerAggregator} + */ + static CombinerAggregator of(R initialValue, + BiFunction accumulator, + BiFunction combiner) { + return new CombinerAggregator() { + @Override + public R init() { + return initialValue; + } + + @Override + public R apply(R aggregate, T value) { + return accumulator.apply(aggregate, value); + } + + @Override + public R merge(R accum1, R accum2) { + return combiner.apply(accum1, accum2); + } + + @Override + public R result(R accum) { + return accum; + } + }; + } + +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/Reducer.java b/storm-core/src/jvm/org/apache/storm/streams/operations/Reducer.java index 04ee70d07c7..48dc6fe0003 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/operations/Reducer.java +++ b/storm-core/src/jvm/org/apache/storm/streams/operations/Reducer.java @@ -23,7 +23,7 @@ * * @param the type of the arguments and the result */ -public interface Reducer extends Operation { +public interface Reducer extends BiFunction { /** * Applies this function to the given arguments. * diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/StateUpdater.java b/storm-core/src/jvm/org/apache/storm/streams/operations/StateUpdater.java new file mode 100644 index 00000000000..a1b13831b1c --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/operations/StateUpdater.java @@ -0,0 +1,67 @@ +/** + * 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.storm.streams.operations; + +/** + * Interface for updating state. + * + * @param the value type + * @param the state type + */ +public interface StateUpdater extends Operation { + /** + * The initial value of the state to start with. + * + * @return the initial value of the state + */ + S init(); + + /** + * Returns a new state by applying the value on the current state. + * + * @param state the current state + * @param value the value + * @return the new state + */ + S apply(S state, T value); + + /** + * A static factory to create a {@link StateUpdater} based on an initial value of the state and a + * state update function. + * + * @param initialValue the intial value of the state + * @param stateUpdateFn the state update function + * @param the value type + * @param the state type + * @return the {@link StateUpdater} + */ + static StateUpdater of(S initialValue, + BiFunction stateUpdateFn) { + return new StateUpdater() { + @Override + public S init() { + return initialValue; + } + + @Override + public S apply(S state, T value) { + return stateUpdateFn.apply(state, value); + } + }; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Count.java b/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Count.java index fd02d2a0247..6589ed6fcd7 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Count.java +++ b/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Count.java @@ -17,21 +17,31 @@ */ package org.apache.storm.streams.operations.aggregators; -import org.apache.storm.streams.operations.Aggregator; +import org.apache.storm.streams.operations.CombinerAggregator; /** * Computes the count of values. * * @param the value type */ -public class Count implements Aggregator { +public class Count implements CombinerAggregator { @Override public Long init() { return 0L; } @Override - public Long apply(T value, Long aggregate) { - return aggregate + 1; + public Long apply(Long accum, T value) { + return accum + 1; + } + + @Override + public Long merge(Long accum1, Long accum2) { + return accum1 + accum2; + } + + @Override + public Long result(Long accum) { + return accum; } } diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Sum.java b/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Sum.java index e23207562d8..df11d996b68 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Sum.java +++ b/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Sum.java @@ -17,19 +17,29 @@ */ package org.apache.storm.streams.operations.aggregators; -import org.apache.storm.streams.operations.Aggregator; +import org.apache.storm.streams.operations.CombinerAggregator; /** * Computes the long sum of the input values */ -public class Sum implements Aggregator { +public class Sum implements CombinerAggregator { @Override public Long init() { return 0L; } @Override - public Long apply(Number value, Long aggregate) { + public Long apply(Long aggregate, Number value) { return value.longValue() + aggregate; } + + @Override + public Long merge(Long accum1, Long accum2) { + return accum1 + accum2; + } + + @Override + public Long result(Long accum) { + return accum; + } } diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/TupleValueMappers.java b/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/TupleValueMappers.java new file mode 100644 index 00000000000..f900f84c249 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/operations/mappers/TupleValueMappers.java @@ -0,0 +1,174 @@ +/** + * 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.storm.streams.operations.mappers; + +import org.apache.storm.streams.tuple.Tuple3; +import org.apache.storm.streams.tuple.Tuple4; +import org.apache.storm.streams.tuple.Tuple5; +import org.apache.storm.streams.tuple.Tuple6; +import org.apache.storm.streams.tuple.Tuple7; +import org.apache.storm.streams.tuple.Tuple8; +import org.apache.storm.streams.tuple.Tuple9; +import org.apache.storm.streams.tuple.Tuple10; +import org.apache.storm.tuple.Tuple; + +/** + * Factory for constructing typed tuples from a {@link Tuple} + * based on indicies. + */ +@SuppressWarnings("unchecked") +public final class TupleValueMappers { + private TupleValueMappers() { + } + + public static + TupleValueMapper> of(int index1, + int index2, + int index3) { + return input -> new Tuple3<>( + (T1) input.getValue(index1), + (T2) input.getValue(index2), + (T3) input.getValue(index3)); + } + + public static + TupleValueMapper> of(int index1, + int index2, + int index3, + int index4) { + return input -> new Tuple4<>( + (T1) input.getValue(index1), + (T2) input.getValue(index2), + (T3) input.getValue(index3), + (T4) input.getValue(index4)); + } + + public static + TupleValueMapper> of(int index1, + int index2, + int index3, + int index4, + int index5) { + return input -> new Tuple5<>( + (T1) input.getValue(index1), + (T2) input.getValue(index2), + (T3) input.getValue(index3), + (T4) input.getValue(index4), + (T5) input.getValue(index5)); + } + + public static + TupleValueMapper> of(int index1, + int index2, + int index3, + int index4, + int index5, + int index6) { + return input -> new Tuple6<>( + (T1) input.getValue(index1), + (T2) input.getValue(index2), + (T3) input.getValue(index3), + (T4) input.getValue(index4), + (T5) input.getValue(index5), + (T6) input.getValue(index6)); + } + + public static + TupleValueMapper> of(int index1, + int index2, + int index3, + int index4, + int index5, + int index6, + int index7) { + return input -> new Tuple7<>( + (T1) input.getValue(index1), + (T2) input.getValue(index2), + (T3) input.getValue(index3), + (T4) input.getValue(index4), + (T5) input.getValue(index5), + (T6) input.getValue(index6), + (T7) input.getValue(index7)); + } + + public static + TupleValueMapper> of(int index1, + int index2, + int index3, + int index4, + int index5, + int index6, + int index7, + int index8) { + return input -> new Tuple8<>( + (T1) input.getValue(index1), + (T2) input.getValue(index2), + (T3) input.getValue(index3), + (T4) input.getValue(index4), + (T5) input.getValue(index5), + (T6) input.getValue(index6), + (T7) input.getValue(index7), + (T8) input.getValue(index8)); + } + + public static + TupleValueMapper> of(int index1, + int index2, + int index3, + int index4, + int index5, + int index6, + int index7, + int index8, + int index9) { + return input -> new Tuple9<>( + (T1) input.getValue(index1), + (T2) input.getValue(index2), + (T3) input.getValue(index3), + (T4) input.getValue(index4), + (T5) input.getValue(index5), + (T6) input.getValue(index6), + (T7) input.getValue(index7), + (T8) input.getValue(index8), + (T9) input.getValue(index9)); + } + + public static + TupleValueMapper> of(int index1, + int index2, + int index3, + int index4, + int index5, + int index6, + int index7, + int index8, + int index9, + int index10) { + return input -> new Tuple10<>( + (T1) input.getValue(index1), + (T2) input.getValue(index2), + (T3) input.getValue(index3), + (T4) input.getValue(index4), + (T5) input.getValue(index5), + (T6) input.getValue(index6), + (T7) input.getValue(index7), + (T8) input.getValue(index8), + (T9) input.getValue(index9), + (T10) input.getValue(index10)); + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateByKeyProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateByKeyProcessor.java index d53a52d4f06..c10113b6965 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateByKeyProcessor.java +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateByKeyProcessor.java @@ -18,37 +18,60 @@ package org.apache.storm.streams.processors; import org.apache.storm.streams.Pair; -import org.apache.storm.streams.operations.Aggregator; +import org.apache.storm.streams.operations.CombinerAggregator; import java.util.HashMap; import java.util.Map; -public class AggregateByKeyProcessor extends BaseProcessor> implements BatchProcessor { - private final Aggregator aggregator; - private final Map state = new HashMap<>(); +public class AggregateByKeyProcessor extends BaseProcessor> implements BatchProcessor { + private final CombinerAggregator aggregator; + private final boolean emitAggregate; + private final Map state = new HashMap<>(); - public AggregateByKeyProcessor(Aggregator aggregator) { + public AggregateByKeyProcessor(CombinerAggregator aggregator) { + this(aggregator, false); + } + + public AggregateByKeyProcessor(CombinerAggregator aggregator, boolean emitAggregate) { this.aggregator = aggregator; + this.emitAggregate = emitAggregate; } @Override public void execute(Pair input) { K key = input.getFirst(); V val = input.getSecond(); - R agg = state.get(key); - if (agg == null) { - agg = aggregator.init(); + A accumulator = state.get(key); + if (accumulator == null) { + accumulator = aggregator.init(); + } + state.put(key, aggregator.apply(accumulator, val)); + if (emitAggregate) { + mayBeForwardAggUpdate(Pair.of(key, state.get(key))); + } else { + mayBeForwardAggUpdate(Pair.of(key, aggregator.result(state.get(key)))); } - state.put(key, aggregator.apply(val, agg)); - mayBeForwardAggUpdate(Pair.of(key, state.get(key))); } @Override public void finish() { - for (Map.Entry entry : state.entrySet()) { - context.forward(Pair.of(entry.getKey(), entry.getValue())); + for (Map.Entry entry : state.entrySet()) { + if (emitAggregate) { + context.forward(Pair.of(entry.getKey(), entry.getValue())); + } else { + context.forward(Pair.of(entry.getKey(), aggregator.result(entry.getValue()))); + } + } state.clear(); } + @Override + public String toString() { + return "AggregateByKeyProcessor{" + + "aggregator=" + aggregator + + ", emitAggregate=" + emitAggregate + + ", state=" + state + + "}"; + } } diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateProcessor.java index c5a1906202a..d1693453c4f 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateProcessor.java +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateProcessor.java @@ -17,14 +17,20 @@ */ package org.apache.storm.streams.processors; -import org.apache.storm.streams.operations.Aggregator; +import org.apache.storm.streams.operations.CombinerAggregator; -public class AggregateProcessor extends BaseProcessor implements BatchProcessor { - private final Aggregator aggregator; - private R state; +public class AggregateProcessor extends BaseProcessor implements BatchProcessor { + private final CombinerAggregator aggregator; + private final boolean emitAggregate; + private A state; - public AggregateProcessor(Aggregator aggregator) { + public AggregateProcessor(CombinerAggregator aggregator) { + this(aggregator, false); + } + + public AggregateProcessor(CombinerAggregator aggregator, boolean emitAggregate) { this.aggregator = aggregator; + this.emitAggregate = emitAggregate; } @Override @@ -32,14 +38,32 @@ public void execute(T input) { if (state == null) { state = aggregator.init(); } - R curAggregate = (state != null) ? state : aggregator.init(); - state = aggregator.apply(input, curAggregate); - mayBeForwardAggUpdate(state); + state = aggregator.apply(state, input); + if (emitAggregate) { + mayBeForwardAggUpdate(state); + } else { + mayBeForwardAggUpdate(aggregator.result(state)); + } } @Override public void finish() { - context.forward(state); - state = null; + if (state != null) { + if (emitAggregate) { + context.forward(state); + } else { + context.forward(aggregator.result(state)); + } + state = null; + } + } + + @Override + public String toString() { + return "AggregateProcessor{" + + "aggregator=" + aggregator + + ", emitAggregate=" + emitAggregate + + ", state=" + state + + "}"; } } diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/BranchProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/BranchProcessor.java index f8bc739d4a9..421a5a7adac 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/processors/BranchProcessor.java +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/BranchProcessor.java @@ -23,15 +23,15 @@ import java.util.Map; public class BranchProcessor extends BaseProcessor { - private final Map, String> predicateToStream = new HashMap<>(); + private final Map, String> predicateToStream = new HashMap<>(); - public void addPredicate(Predicate predicate, String stream) { + public void addPredicate(Predicate predicate, String stream) { predicateToStream.put(predicate, stream); } @Override public void execute(T input) { - for (Map.Entry, String> entry : predicateToStream.entrySet()) { + for (Map.Entry, String> entry : predicateToStream.entrySet()) { if (entry.getKey().test(input)) { context.forward(input, entry.getValue()); break; diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/EmittingProcessorContext.java b/storm-core/src/jvm/org/apache/storm/streams/processors/EmittingProcessorContext.java index d841080db3b..a85eaf7315b 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/processors/EmittingProcessorContext.java +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/EmittingProcessorContext.java @@ -46,11 +46,11 @@ public class EmittingProcessorContext implements ProcessorContext { private static final Logger LOG = LoggerFactory.getLogger(EmittingProcessorContext.class); private final ProcessorNode processorNode; private final String outputStreamId; + private final String punctuationStreamId; private final OutputCollector collector; private final Fields outputFields; private final Values punctuation; private final List anchors = new ArrayList<>(); - private boolean emitPunctuation = true; private long eventTimestamp; private String timestampField; @@ -59,23 +59,20 @@ public EmittingProcessorContext(ProcessorNode processorNode, OutputCollector col this.outputStreamId = outputStreamId; this.collector = collector; outputFields = processorNode.getOutputFields(); - punctuation = createPunctuation(); + punctuation = new Values(PUNCTUATION); + punctuationStreamId = StreamUtil.getPunctuationStream(outputStreamId); } @Override public void forward(T input) { - if (input instanceof Pair) { - Pair value = (Pair) input; - emit(new Values(value.getFirst(), value.getSecond())); - } else if (PUNCTUATION.equals(input)) { - if (emitPunctuation) { - emit(punctuation); - } else { - LOG.debug("Not emitting punctuation since emitPunctuation is false"); - } + if (PUNCTUATION.equals(input)) { + emit(punctuation, punctuationStreamId); maybeAck(); + } else if (processorNode.emitsPair()) { + Pair value = (Pair) input; + emit(new Values(value.getFirst(), value.getSecond()), outputStreamId); } else { - emit(new Values(input)); + emit(new Values(input), outputStreamId); } } @@ -96,10 +93,6 @@ public Set getWindowedParentStreams() { return processorNode.getWindowedParentStreams(); } - public void setEmitPunctuation(boolean emitPunctuation) { - this.emitPunctuation = emitPunctuation; - } - public void setTimestampField(String fieldName) { timestampField = fieldName; } @@ -128,14 +121,6 @@ public void setEventTimestamp(long timestamp) { this.eventTimestamp = timestamp; } - private Values createPunctuation() { - Values values = new Values(); - for (int i = 0; i < outputFields.size(); i++) { - values.add(PUNCTUATION); - } - return values; - } - private void maybeAck() { if (!anchors.isEmpty()) { for (RefCountedTuple anchor : anchors) { @@ -154,7 +139,7 @@ private Collection tuples(Collection anchors) { return anchors.stream().map(RefCountedTuple::tuple).collect(Collectors.toList()); } - private void emit(Values values) { + private void emit(Values values, String outputStreamId) { if (timestampField != null) { values.add(eventTimestamp); } diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/JoinProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/JoinProcessor.java index d56cfea9ad1..05cad8d750b 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/processors/JoinProcessor.java +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/JoinProcessor.java @@ -20,12 +20,14 @@ import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Multimap; import org.apache.storm.streams.Pair; -import org.apache.storm.streams.Tuple3; import org.apache.storm.streams.operations.ValueJoiner; +import org.apache.storm.streams.tuple.Tuple3; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Map; /** * Provides equi-join implementation based on simple hash-join. @@ -36,11 +38,25 @@ public class JoinProcessor extends BaseProcessor> imple private final String rightStream; private final List> leftRows = new ArrayList<>(); private final List> rightRows = new ArrayList<>(); + private final JoinType leftType; + private final JoinType rightType; + + public enum JoinType { + INNER, + OUTER + } public JoinProcessor(String leftStream, String rightStream, ValueJoiner valueJoiner) { + this(leftStream, rightStream, valueJoiner, JoinType.INNER, JoinType.INNER); + } + + public JoinProcessor(String leftStream, String rightStream, ValueJoiner valueJoiner, + JoinType leftType, JoinType rightType) { this.valueJoiner = valueJoiner; this.leftStream = leftStream; this.rightStream = rightStream; + this.leftType = leftType; + this.rightType = rightType; } @Override @@ -78,30 +94,54 @@ public String getRightStream() { return rightStream; } + /* + * performs a hash-join by constructing a hash map of the smaller set, iterating over the + * larger set and finding matching rows in the hash map. + */ private void joinAndForward(List> leftRows, List> rightRows) { - if (leftRows.size() <= rightRows.size()) { - for (Tuple3 res : join(getJoinTable(leftRows), rightRows)) { + if (leftRows.size() < rightRows.size()) { + for (Tuple3 res : join(getJoinTable(leftRows), rightRows, leftType, rightType)) { context.forward(Pair.of(res._1, valueJoiner.apply(res._2, res._3))); } } else { - for (Tuple3 res : join(getJoinTable(rightRows), leftRows)) { + for (Tuple3 res : join(getJoinTable(rightRows), leftRows, rightType, leftType)) { context.forward(Pair.of(res._1, valueJoiner.apply(res._3, res._2))); } } } - private List> join(Multimap tab, List> rows) { + /* + * returns list of Tuple3 (key, val from table, val from row) + */ + private List> join(Multimap tab, List> rows, + JoinType leftType, JoinType rightType) { List> res = new ArrayList<>(); for (Pair row : rows) { - for (T1 mapValue : tab.get(row.getFirst())) { - if (mapValue != null) { + K key = row.getFirst(); + Collection values = tab.removeAll(key); + if (values.isEmpty()) { + if (rightType == JoinType.OUTER) { + res.add(new Tuple3<>(row.getFirst(), null, row.getSecond())); + } + } else { + for (T1 mapValue : values) { res.add(new Tuple3<>(row.getFirst(), mapValue, row.getSecond())); } } } + // whatever remains in the tab are non matching left rows. + if (leftType == JoinType.OUTER) { + for (Map.Entry row : tab.entries()) { + res.add(new Tuple3<>(row.getKey(), row.getValue(), null)); + } + } return res; } + /* + * key1 -> (val1, val2 ..) + * key2 -> (val3, val4 ..) + */ private Multimap getJoinTable(List> rows) { Multimap m = ArrayListMultimap.create(); for (Pair v : rows) { diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateByKeyProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateByKeyProcessor.java new file mode 100644 index 00000000000..57ad8454b55 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateByKeyProcessor.java @@ -0,0 +1,54 @@ +/** + * 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.storm.streams.processors; + +import org.apache.storm.streams.Pair; +import org.apache.storm.streams.operations.CombinerAggregator; + +import java.util.HashMap; +import java.util.Map; + +public class MergeAggregateByKeyProcessor extends BaseProcessor> implements BatchProcessor { + protected final CombinerAggregator aggregator; + protected final Map state = new HashMap<>(); + + public MergeAggregateByKeyProcessor(CombinerAggregator aggregator) { + this.aggregator = aggregator; + } + + @Override + public void execute(Pair input) { + K key = input.getFirst(); + A val = input.getSecond(); + A accumulator = state.get(key); + if (accumulator == null) { + accumulator = aggregator.init(); + } + state.put(key, aggregator.merge(accumulator, val)); + mayBeForwardAggUpdate(Pair.of(key, aggregator.result(state.get(key)))); + } + + @Override + public void finish() { + for (Map.Entry entry : state.entrySet()) { + context.forward(Pair.of(entry.getKey(), aggregator.result(entry.getValue()))); + } + state.clear(); + } + +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateProcessor.java new file mode 100644 index 00000000000..61b555b4c78 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateProcessor.java @@ -0,0 +1,47 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.streams.processors; + +import org.apache.storm.streams.operations.CombinerAggregator; + +public class MergeAggregateProcessor extends BaseProcessor implements BatchProcessor { + private final CombinerAggregator aggregator; + private A state; + + public MergeAggregateProcessor(CombinerAggregator aggregator) { + this.aggregator = aggregator; + } + + @Override + protected void execute(A input) { + if (state == null) { + state = aggregator.init(); + } + state = aggregator.merge(state, input); + mayBeForwardAggUpdate(aggregator.result(state)); + } + + @Override + public void finish() { + if (state != null) { + context.forward(aggregator.result(state)); + state = null; + } + } + +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceProcessor.java index d64e1142036..0b90fb9110e 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceProcessor.java +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceProcessor.java @@ -35,7 +35,9 @@ public void execute(T input) { @Override public void finish() { - context.forward(agg); - agg = null; + if (agg != null) { + context.forward(agg); + agg = null; + } } } diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/UpdateStateByKeyProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/UpdateStateByKeyProcessor.java index 9e068a08ca3..5fb27307195 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/processors/UpdateStateByKeyProcessor.java +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/UpdateStateByKeyProcessor.java @@ -19,14 +19,14 @@ import org.apache.storm.state.KeyValueState; import org.apache.storm.streams.Pair; -import org.apache.storm.streams.operations.Aggregator; +import org.apache.storm.streams.operations.StateUpdater; public class UpdateStateByKeyProcessor extends BaseProcessor> implements StatefulProcessor { - private final Aggregator aggregator; + private final StateUpdater stateUpdater; private KeyValueState keyValueState; - public UpdateStateByKeyProcessor(Aggregator aggregator) { - this.aggregator = aggregator; + public UpdateStateByKeyProcessor(StateUpdater stateUpdater) { + this.stateUpdater = stateUpdater; } @Override @@ -40,9 +40,9 @@ protected void execute(Pair input) { V val = input.getSecond(); R agg = keyValueState.get(key); if (agg == null) { - agg = aggregator.init(); + agg = stateUpdater.init(); } - R newAgg = aggregator.apply(val, agg); + R newAgg = stateUpdater.apply(agg, val); keyValueState.put(key, newAgg); context.forward(Pair.of(key, newAgg)); } diff --git a/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple10.java b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple10.java new file mode 100644 index 00000000000..879d71c9bbf --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple10.java @@ -0,0 +1,112 @@ +/** + * 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.storm.streams.tuple; + +/** + * A tuple of ten elements along the lines of Scala's Tuple. + * + * @param the type of the first element + * @param the type of the second element + * @param the type of the third element + * @param the type of the fourth element + * @param the type of the fifth element + * @param the type of the sixth element + * @param the type of the seventh element + * @param the type of the eighth element + * @param the type of the ninth element + * @param the type of the tenth element + */ +public class Tuple10 { + public final T1 _1; + public final T2 _2; + public final T3 _3; + public final T4 _4; + public final T5 _5; + public final T6 _6; + public final T7 _7; + public final T8 _8; + public final T9 _9; + public final T10 _10; + + /** + * Constructs a new tuple. + * + * @param _1 the first element + * @param _2 the second element + * @param _3 the third element + * @param _4 the fourth element + * @param _5 the fifth element + * @param _6 the sixth element + * @param _7 the seventh element + * @param _8 the eighth element + * @param _9 the ninth element + * @param _10 the tenth element + */ + public Tuple10(T1 _1, T2 _2, T3 _3, T4 _4, T5 _5, T6 _6, T7 _7, T8 _8, T9 _9, T10 _10) { + this._1 = _1; + this._2 = _2; + this._3 = _3; + this._4 = _4; + this._5 = _5; + this._6 = _6; + this._7 = _7; + this._8 = _8; + this._9 = _9; + this._10 = _10; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + Tuple10 tuple10 = (Tuple10) o; + + if (_1 != null ? !_1.equals(tuple10._1) : tuple10._1 != null) return false; + if (_2 != null ? !_2.equals(tuple10._2) : tuple10._2 != null) return false; + if (_3 != null ? !_3.equals(tuple10._3) : tuple10._3 != null) return false; + if (_4 != null ? !_4.equals(tuple10._4) : tuple10._4 != null) return false; + if (_5 != null ? !_5.equals(tuple10._5) : tuple10._5 != null) return false; + if (_6 != null ? !_6.equals(tuple10._6) : tuple10._6 != null) return false; + if (_7 != null ? !_7.equals(tuple10._7) : tuple10._7 != null) return false; + if (_8 != null ? !_8.equals(tuple10._8) : tuple10._8 != null) return false; + if (_9 != null ? !_9.equals(tuple10._9) : tuple10._9 != null) return false; + return _10 != null ? _10.equals(tuple10._10) : tuple10._10 == null; + + } + + @Override + public int hashCode() { + int result = _1 != null ? _1.hashCode() : 0; + result = 31 * result + (_2 != null ? _2.hashCode() : 0); + result = 31 * result + (_3 != null ? _3.hashCode() : 0); + result = 31 * result + (_4 != null ? _4.hashCode() : 0); + result = 31 * result + (_5 != null ? _5.hashCode() : 0); + result = 31 * result + (_6 != null ? _6.hashCode() : 0); + result = 31 * result + (_7 != null ? _7.hashCode() : 0); + result = 31 * result + (_8 != null ? _8.hashCode() : 0); + result = 31 * result + (_9 != null ? _9.hashCode() : 0); + result = 31 * result + (_10 != null ? _10.hashCode() : 0); + return result; + } + + @Override + public String toString() { + return "(" + _1 + "," + _2 + "," + _3 + "," + _4 + "," + _5 + "," + _6 + "," + _7 + "," + _8 + "," + _9 + "," + _10 + ")"; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/Tuple3.java b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple3.java similarity index 63% rename from storm-core/src/jvm/org/apache/storm/streams/Tuple3.java rename to storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple3.java index 77973f2746b..514e16931f7 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/Tuple3.java +++ b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple3.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.storm.streams; +package org.apache.storm.streams.tuple; /** * A tuple of three elements along the lines of Scala's Tuple. @@ -24,13 +24,13 @@ * @param the type of the second element * @param the type of the third element */ -public class Tuple3 { +public final class Tuple3 { public final T1 _1; public final T2 _2; public final T3 _3; /** - * Constructs a new tuple of three elements. + * Constructs a new tuple. * * @param _1 the first element * @param _2 the second element @@ -42,6 +42,27 @@ public Tuple3(T1 _1, T2 _2, T3 _3) { this._3 = _3; } + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + Tuple3 tuple3 = (Tuple3) o; + + if (_1 != null ? !_1.equals(tuple3._1) : tuple3._1 != null) return false; + if (_2 != null ? !_2.equals(tuple3._2) : tuple3._2 != null) return false; + return _3 != null ? _3.equals(tuple3._3) : tuple3._3 == null; + + } + + @Override + public int hashCode() { + int result = _1 != null ? _1.hashCode() : 0; + result = 31 * result + (_2 != null ? _2.hashCode() : 0); + result = 31 * result + (_3 != null ? _3.hashCode() : 0); + return result; + } + @Override public String toString() { return "(" + _1 + "," + _2 + "," + _3 + ")"; diff --git a/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple4.java b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple4.java new file mode 100644 index 00000000000..5ae3fb1bf59 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple4.java @@ -0,0 +1,76 @@ +/** + * 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.storm.streams.tuple; + +/** + * A tuple of four elements along the lines of Scala's Tuple. + * + * @param the type of the first element + * @param the type of the second element + * @param the type of the third element + * @param the type of the fourth element + */ +public final class Tuple4 { + public final T1 _1; + public final T2 _2; + public final T3 _3; + public final T4 _4; + + /** + * Constructs a new tuple. + * + * @param _1 the first element + * @param _2 the second element + * @param _3 the third element + * @param _4 the fourth element + */ + public Tuple4(T1 _1, T2 _2, T3 _3, T4 _4) { + this._1 = _1; + this._2 = _2; + this._3 = _3; + this._4 = _4; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + Tuple4 tuple4 = (Tuple4) o; + + if (_1 != null ? !_1.equals(tuple4._1) : tuple4._1 != null) return false; + if (_2 != null ? !_2.equals(tuple4._2) : tuple4._2 != null) return false; + if (_3 != null ? !_3.equals(tuple4._3) : tuple4._3 != null) return false; + return _4 != null ? _4.equals(tuple4._4) : tuple4._4 == null; + + } + + @Override + public int hashCode() { + int result = _1 != null ? _1.hashCode() : 0; + result = 31 * result + (_2 != null ? _2.hashCode() : 0); + result = 31 * result + (_3 != null ? _3.hashCode() : 0); + result = 31 * result + (_4 != null ? _4.hashCode() : 0); + return result; + } + + @Override + public String toString() { + return "(" + _1 + "," + _2 + "," + _3 + "," + _4 + ")"; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple5.java b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple5.java new file mode 100644 index 00000000000..6b0f81e42a9 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple5.java @@ -0,0 +1,82 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.streams.tuple; + +/** + * A tuple of five elements along the lines of Scala's Tuple. + * + * @param the type of the first element + * @param the type of the second element + * @param the type of the third element + * @param the type of the fourth element + * @param the type of the fifth element + */ +public class Tuple5 { + public final T1 _1; + public final T2 _2; + public final T3 _3; + public final T4 _4; + public final T5 _5; + + /** + * Constructs a new tuple. + * + * @param _1 the first element + * @param _2 the second element + * @param _3 the third element + * @param _4 the fourth element + * @param _5 the fifth element + */ + public Tuple5(T1 _1, T2 _2, T3 _3, T4 _4, T5 _5) { + this._1 = _1; + this._2 = _2; + this._3 = _3; + this._4 = _4; + this._5 = _5; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + Tuple5 tuple5 = (Tuple5) o; + + if (_1 != null ? !_1.equals(tuple5._1) : tuple5._1 != null) return false; + if (_2 != null ? !_2.equals(tuple5._2) : tuple5._2 != null) return false; + if (_3 != null ? !_3.equals(tuple5._3) : tuple5._3 != null) return false; + if (_4 != null ? !_4.equals(tuple5._4) : tuple5._4 != null) return false; + return _5 != null ? _5.equals(tuple5._5) : tuple5._5 == null; + + } + + @Override + public int hashCode() { + int result = _1 != null ? _1.hashCode() : 0; + result = 31 * result + (_2 != null ? _2.hashCode() : 0); + result = 31 * result + (_3 != null ? _3.hashCode() : 0); + result = 31 * result + (_4 != null ? _4.hashCode() : 0); + result = 31 * result + (_5 != null ? _5.hashCode() : 0); + return result; + } + + @Override + public String toString() { + return "(" + _1 + "," + _2 + "," + _3 + "," + _4 + "," + _5 + ")"; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple6.java b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple6.java new file mode 100644 index 00000000000..4c35e27414d --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple6.java @@ -0,0 +1,89 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.streams.tuple; + + +/** + * A tuple of six elements along the lines of Scala's Tuple. + * + * @param the type of the first element + * @param the type of the second element + * @param the type of the third element + * @param the type of the fourth element + * @param the type of the fifth element + * @param the type of the sixth element + */ +public class Tuple6 { + public final T1 _1; + public final T2 _2; + public final T3 _3; + public final T4 _4; + public final T5 _5; + public final T6 _6; + + /** + * Constructs a new tuple. + * + * @param _1 the first element + * @param _2 the second element + * @param _3 the third element + * @param _4 the fourth element + * @param _5 the fifth element + * @param _6 the sixth element + */ + public Tuple6(T1 _1, T2 _2, T3 _3, T4 _4, T5 _5, T6 _6) { + this._1 = _1; + this._2 = _2; + this._3 = _3; + this._4 = _4; + this._5 = _5; + this._6 = _6; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + Tuple6 tuple6 = (Tuple6) o; + + if (_1 != null ? !_1.equals(tuple6._1) : tuple6._1 != null) return false; + if (_2 != null ? !_2.equals(tuple6._2) : tuple6._2 != null) return false; + if (_3 != null ? !_3.equals(tuple6._3) : tuple6._3 != null) return false; + if (_4 != null ? !_4.equals(tuple6._4) : tuple6._4 != null) return false; + if (_5 != null ? !_5.equals(tuple6._5) : tuple6._5 != null) return false; + return _6 != null ? _6.equals(tuple6._6) : tuple6._6 == null; + + } + + @Override + public int hashCode() { + int result = _1 != null ? _1.hashCode() : 0; + result = 31 * result + (_2 != null ? _2.hashCode() : 0); + result = 31 * result + (_3 != null ? _3.hashCode() : 0); + result = 31 * result + (_4 != null ? _4.hashCode() : 0); + result = 31 * result + (_5 != null ? _5.hashCode() : 0); + result = 31 * result + (_6 != null ? _6.hashCode() : 0); + return result; + } + + @Override + public String toString() { + return "(" + _1 + "," + _2 + "," + _3 + "," + _4 + "," + _5 + "," + _6 + ")"; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple7.java b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple7.java new file mode 100644 index 00000000000..366e8e9413d --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple7.java @@ -0,0 +1,94 @@ +/** + * 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.storm.streams.tuple; + +/** + * A tuple of seven elements along the lines of Scala's Tuple. + * + * @param the type of the first element + * @param the type of the second element + * @param the type of the third element + * @param the type of the fourth element + * @param the type of the fifth element + * @param the type of the sixth element + * @param the type of the seventh element + */ +public class Tuple7 { + public final T1 _1; + public final T2 _2; + public final T3 _3; + public final T4 _4; + public final T5 _5; + public final T6 _6; + public final T7 _7; + + /** + * Constructs a new tuple. + * + * @param _1 the first element + * @param _2 the second element + * @param _3 the third element + * @param _4 the fourth element + * @param _5 the fifth element + * @param _6 the sixth element + * @param _7 the seventh element + */ + public Tuple7(T1 _1, T2 _2, T3 _3, T4 _4, T5 _5, T6 _6, T7 _7) { + this._1 = _1; + this._2 = _2; + this._3 = _3; + this._4 = _4; + this._5 = _5; + this._6 = _6; + this._7 = _7; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + Tuple7 tuple7 = (Tuple7) o; + + if (_1 != null ? !_1.equals(tuple7._1) : tuple7._1 != null) return false; + if (_2 != null ? !_2.equals(tuple7._2) : tuple7._2 != null) return false; + if (_3 != null ? !_3.equals(tuple7._3) : tuple7._3 != null) return false; + if (_4 != null ? !_4.equals(tuple7._4) : tuple7._4 != null) return false; + if (_5 != null ? !_5.equals(tuple7._5) : tuple7._5 != null) return false; + if (_6 != null ? !_6.equals(tuple7._6) : tuple7._6 != null) return false; + return _7 != null ? _7.equals(tuple7._7) : tuple7._7 == null; + + } + + @Override + public int hashCode() { + int result = _1 != null ? _1.hashCode() : 0; + result = 31 * result + (_2 != null ? _2.hashCode() : 0); + result = 31 * result + (_3 != null ? _3.hashCode() : 0); + result = 31 * result + (_4 != null ? _4.hashCode() : 0); + result = 31 * result + (_5 != null ? _5.hashCode() : 0); + result = 31 * result + (_6 != null ? _6.hashCode() : 0); + result = 31 * result + (_7 != null ? _7.hashCode() : 0); + return result; + } + + @Override + public String toString() { + return "(" + _1 + "," + _2 + "," + _3 + "," + _4 + "," + _5 + "," + _6 + "," + _7 + ")"; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple8.java b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple8.java new file mode 100644 index 00000000000..bf088df11c8 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple8.java @@ -0,0 +1,100 @@ +/** + * 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.storm.streams.tuple; + +/** + * A tuple of eight elements along the lines of Scala's Tuple. + * + * @param the type of the first element + * @param the type of the second element + * @param the type of the third element + * @param the type of the fourth element + * @param the type of the fifth element + * @param the type of the sixth element + * @param the type of the seventh element + * @param the type of the eighth element + */ +public class Tuple8 { + public final T1 _1; + public final T2 _2; + public final T3 _3; + public final T4 _4; + public final T5 _5; + public final T6 _6; + public final T7 _7; + public final T8 _8; + + /** + * Constructs a new tuple. + * + * @param _1 the first element + * @param _2 the second element + * @param _3 the third element + * @param _4 the fourth element + * @param _5 the fifth element + * @param _6 the sixth element + * @param _7 the seventh element + * @param _8 the eighth element + */ + public Tuple8(T1 _1, T2 _2, T3 _3, T4 _4, T5 _5, T6 _6, T7 _7, T8 _8) { + this._1 = _1; + this._2 = _2; + this._3 = _3; + this._4 = _4; + this._5 = _5; + this._6 = _6; + this._7 = _7; + this._8 = _8; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + Tuple8 tuple8 = (Tuple8) o; + + if (_1 != null ? !_1.equals(tuple8._1) : tuple8._1 != null) return false; + if (_2 != null ? !_2.equals(tuple8._2) : tuple8._2 != null) return false; + if (_3 != null ? !_3.equals(tuple8._3) : tuple8._3 != null) return false; + if (_4 != null ? !_4.equals(tuple8._4) : tuple8._4 != null) return false; + if (_5 != null ? !_5.equals(tuple8._5) : tuple8._5 != null) return false; + if (_6 != null ? !_6.equals(tuple8._6) : tuple8._6 != null) return false; + if (_7 != null ? !_7.equals(tuple8._7) : tuple8._7 != null) return false; + return _8 != null ? _8.equals(tuple8._8) : tuple8._8 == null; + + } + + @Override + public int hashCode() { + int result = _1 != null ? _1.hashCode() : 0; + result = 31 * result + (_2 != null ? _2.hashCode() : 0); + result = 31 * result + (_3 != null ? _3.hashCode() : 0); + result = 31 * result + (_4 != null ? _4.hashCode() : 0); + result = 31 * result + (_5 != null ? _5.hashCode() : 0); + result = 31 * result + (_6 != null ? _6.hashCode() : 0); + result = 31 * result + (_7 != null ? _7.hashCode() : 0); + result = 31 * result + (_8 != null ? _8.hashCode() : 0); + return result; + } + + @Override + public String toString() { + return "(" + _1 + "," + _2 + "," + _3 + "," + _4 + "," + _5 + "," + _6 + "," + _7 + "," + _8 + ")"; + } +} diff --git a/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple9.java b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple9.java new file mode 100644 index 00000000000..0cd7139639a --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/streams/tuple/Tuple9.java @@ -0,0 +1,106 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.storm.streams.tuple; + +/** + * A tuple of nine elements along the lines of Scala's Tuple. + * + * @param the type of the first element + * @param the type of the second element + * @param the type of the third element + * @param the type of the fourth element + * @param the type of the fifth element + * @param the type of the sixth element + * @param the type of the seventh element + * @param the type of the eighth element + * @param the type of the ninth element + */ +public class Tuple9 { + public final T1 _1; + public final T2 _2; + public final T3 _3; + public final T4 _4; + public final T5 _5; + public final T6 _6; + public final T7 _7; + public final T8 _8; + public final T9 _9; + + /** + * Constructs a new tuple. + * + * @param _1 the first element + * @param _2 the second element + * @param _3 the third element + * @param _4 the fourth element + * @param _5 the fifth element + * @param _6 the sixth element + * @param _7 the seventh element + * @param _8 the eighth element + * @param _9 the ninth element + */ + public Tuple9(T1 _1, T2 _2, T3 _3, T4 _4, T5 _5, T6 _6, T7 _7, T8 _8, T9 _9) { + this._1 = _1; + this._2 = _2; + this._3 = _3; + this._4 = _4; + this._5 = _5; + this._6 = _6; + this._7 = _7; + this._8 = _8; + this._9 = _9; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + Tuple9 tuple9 = (Tuple9) o; + + if (_1 != null ? !_1.equals(tuple9._1) : tuple9._1 != null) return false; + if (_2 != null ? !_2.equals(tuple9._2) : tuple9._2 != null) return false; + if (_3 != null ? !_3.equals(tuple9._3) : tuple9._3 != null) return false; + if (_4 != null ? !_4.equals(tuple9._4) : tuple9._4 != null) return false; + if (_5 != null ? !_5.equals(tuple9._5) : tuple9._5 != null) return false; + if (_6 != null ? !_6.equals(tuple9._6) : tuple9._6 != null) return false; + if (_7 != null ? !_7.equals(tuple9._7) : tuple9._7 != null) return false; + if (_8 != null ? !_8.equals(tuple9._8) : tuple9._8 != null) return false; + return _9 != null ? _9.equals(tuple9._9) : tuple9._9 == null; + + } + + @Override + public int hashCode() { + int result = _1 != null ? _1.hashCode() : 0; + result = 31 * result + (_2 != null ? _2.hashCode() : 0); + result = 31 * result + (_3 != null ? _3.hashCode() : 0); + result = 31 * result + (_4 != null ? _4.hashCode() : 0); + result = 31 * result + (_5 != null ? _5.hashCode() : 0); + result = 31 * result + (_6 != null ? _6.hashCode() : 0); + result = 31 * result + (_7 != null ? _7.hashCode() : 0); + result = 31 * result + (_8 != null ? _8.hashCode() : 0); + result = 31 * result + (_9 != null ? _9.hashCode() : 0); + return result; + } + + @Override + public String toString() { + return "(" + _1 + "," + _2 + "," + _3 + "," + _4 + "," + _5 + "," + _6 + "," + _7 + "," + _8 + "," + _9 + ")"; + } +} diff --git a/storm-core/test/jvm/org/apache/storm/streams/ProcessorBoltTest.java b/storm-core/test/jvm/org/apache/storm/streams/ProcessorBoltTest.java index e9d51275d9f..aa877f92385 100644 --- a/storm-core/test/jvm/org/apache/storm/streams/ProcessorBoltTest.java +++ b/storm-core/test/jvm/org/apache/storm/streams/ProcessorBoltTest.java @@ -18,6 +18,7 @@ package org.apache.storm.streams; import com.google.common.collect.Multimap; +import org.apache.storm.generated.GlobalStreamId; import org.apache.storm.streams.operations.aggregators.Sum; import org.apache.storm.streams.processors.AggregateProcessor; import org.apache.storm.streams.processors.FilterProcessor; @@ -34,10 +35,10 @@ import org.mockito.ArgumentCaptor; import org.mockito.Mockito; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.Map; import java.util.Set; import static org.junit.Assert.assertArrayEquals; @@ -99,7 +100,7 @@ public void testAggResultAndAck() throws Exception { assertArrayEquals(new Object[]{mockTuple2, mockTuple3, punctuation}, anchor.getAllValues().get(0).toArray()); assertArrayEquals(new Object[]{mockTuple2, mockTuple3, punctuation}, anchor.getAllValues().get(1).toArray()); assertArrayEquals(new Object[]{new Values(200L), new Values("__punctuation")}, values.getAllValues().toArray()); - assertArrayEquals(new Object[]{"outputstream", "outputstream"}, os.getAllValues().toArray()); + assertArrayEquals(new Object[]{"outputstream", "outputstream__punctuation"}, os.getAllValues().toArray()); Mockito.verify(mockOutputCollector).ack(mockTuple2); Mockito.verify(mockOutputCollector).ack(mockTuple3); Mockito.verify(mockOutputCollector).ack(punctuation); @@ -138,6 +139,14 @@ private void setUpProcessorBolt(Processor processor, node.setWindowedParentStreams(windowedParentStreams); node.setWindowed(isWindowed); Mockito.when(mockStreamToProcessors.get(Mockito.anyString())).thenReturn(Collections.singletonList(node)); + Mockito.when(mockStreamToProcessors.keySet()).thenReturn(Collections.singleton("inputstream")); + Map mockSources = Mockito.mock(Map.class); + GlobalStreamId mockGlobalStreamId = Mockito.mock(GlobalStreamId.class); + Mockito.when(mockTopologyContext.getThisSources()).thenReturn(mockSources); + Mockito.when(mockSources.keySet()).thenReturn(Collections.singleton(mockGlobalStreamId)); + Mockito.when(mockGlobalStreamId.get_streamId()).thenReturn("inputstream"); + Mockito.when(mockGlobalStreamId.get_componentId()).thenReturn("bolt0"); + Mockito.when(mockTopologyContext.getComponentTasks(Mockito.anyString())).thenReturn(Collections.singletonList(1)); graph.addVertex(node); bolt = new ProcessorBolt("bolt1", graph, Collections.singletonList(node)); if (tsFieldName != null && !tsFieldName.isEmpty()) { diff --git a/storm-core/test/jvm/org/apache/storm/streams/StatefulProcessorBoltTest.java b/storm-core/test/jvm/org/apache/storm/streams/StatefulProcessorBoltTest.java index dbc7e27edf9..2e6e54aabec 100644 --- a/storm-core/test/jvm/org/apache/storm/streams/StatefulProcessorBoltTest.java +++ b/storm-core/test/jvm/org/apache/storm/streams/StatefulProcessorBoltTest.java @@ -19,7 +19,7 @@ import com.google.common.collect.Multimap; import org.apache.storm.state.KeyValueState; -import org.apache.storm.streams.operations.aggregators.Count; +import org.apache.storm.streams.operations.StateUpdater; import org.apache.storm.streams.processors.Processor; import org.apache.storm.streams.processors.UpdateStateByKeyProcessor; import org.apache.storm.task.OutputCollector; @@ -65,7 +65,17 @@ public void setUp() throws Exception { @Test public void testEmitAndAck() throws Exception { - setUpStatefulProcessorBolt(new UpdateStateByKeyProcessor<>(new Count<>())); + setUpStatefulProcessorBolt(new UpdateStateByKeyProcessor<>(new StateUpdater() { + @Override + public Long init() { + return 0L; + } + + @Override + public Long apply(Long state, Object value) { + return state + 1; + } + })); bolt.execute(mockTuple1); ArgumentCaptor anchor = ArgumentCaptor.forClass(Collection.class); ArgumentCaptor values = ArgumentCaptor.forClass(Values.class); @@ -80,6 +90,7 @@ public void testEmitAndAck() throws Exception { private void setUpStatefulProcessorBolt(Processor processor) { ProcessorNode node = new ProcessorNode(processor, "outputstream", new Fields("value")); + node.setEmitsPair(true); Mockito.when(mockStreamToProcessors.get(Mockito.anyString())).thenReturn(Collections.singletonList(node)); graph = new DefaultDirectedGraph(new StreamsEdgeFactory()); graph.addVertex(node); diff --git a/storm-core/test/jvm/org/apache/storm/streams/StreamBuilderTest.java b/storm-core/test/jvm/org/apache/storm/streams/StreamBuilderTest.java index 1498ae4bff6..c88fe343e08 100644 --- a/storm-core/test/jvm/org/apache/storm/streams/StreamBuilderTest.java +++ b/storm-core/test/jvm/org/apache/storm/streams/StreamBuilderTest.java @@ -28,6 +28,7 @@ import org.apache.storm.streams.operations.mappers.PairValueMapper; import org.apache.storm.streams.operations.mappers.ValueMapper; import org.apache.storm.streams.processors.BranchProcessor; +import org.apache.storm.streams.windowing.TumblingWindows; import org.apache.storm.task.OutputCollector; import org.apache.storm.task.TopologyContext; import org.apache.storm.topology.IRichBolt; @@ -35,6 +36,7 @@ import org.apache.storm.topology.OutputFieldsDeclarer; import org.apache.storm.topology.base.BaseRichBolt; import org.apache.storm.topology.base.BaseRichSpout; +import org.apache.storm.topology.base.BaseWindowedBolt; import org.apache.storm.tuple.Fields; import org.apache.storm.tuple.Tuple; import org.apache.storm.utils.Utils; @@ -115,9 +117,9 @@ public void testJoin() throws Exception { @Test public void testGroupBy() throws Exception { - PairStream stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new PairValueMapper<>(0, 1)); + PairStream stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new PairValueMapper<>(0, 1), 2); - stream.groupByKey().aggregateByKey(new Count<>()); + stream.window(TumblingWindows.of(BaseWindowedBolt.Count.of(10))).aggregateByKey(new Count<>()); StormTopology topology = streamBuilder.build(); assertEquals(2, topology.get_bolts_size()); @@ -129,7 +131,7 @@ public void testGroupBy() throws Exception { @Test public void testGlobalAggregate() throws Exception { - Stream stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new ValueMapper<>(0)); + Stream stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new ValueMapper<>(0), 2); stream.aggregate(new Count<>()); @@ -142,6 +144,7 @@ public void testGlobalAggregate() throws Exception { expected1.put(new GlobalStreamId(spoutId, "default"), Grouping.shuffle(new NullStruct())); Map expected2 = new HashMap<>(); expected2.put(new GlobalStreamId("bolt1", "s1"), Grouping.fields(Collections.emptyList())); + expected2.put(new GlobalStreamId("bolt1", "s1__punctuation"), Grouping.all(new NullStruct())); assertEquals(expected1, bolt1.get_common().get_inputs()); assertEquals(expected2, bolt2.get_common().get_inputs()); } @@ -169,11 +172,57 @@ public void testRepartition() throws Exception { public void testBranchAndJoin() throws Exception { TopologyContext mockContext = Mockito.mock(TopologyContext.class); OutputCollector mockCollector = Mockito.mock(OutputCollector.class); - Stream stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new ValueMapper<>(0)); + Stream stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new ValueMapper<>(0), 2); Stream[] streams = stream.branch(x -> x % 2 == 0, x -> x % 2 == 1); PairStream> joined = streams[0].mapToPair(x -> Pair.of(x, 1)).join(streams[1].mapToPair(x -> Pair.of(x, 1))); assertTrue(joined.getNode() instanceof ProcessorNode); StormTopology topology = streamBuilder.build(); + assertEquals(2, topology.get_bolts_size()); + } + + @Test + public void testMultiPartitionByKey() { + TopologyContext mockContext = Mockito.mock(TopologyContext.class); + OutputCollector mockCollector = Mockito.mock(OutputCollector.class); + Stream stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new ValueMapper<>(0)); + stream.mapToPair(x -> Pair.of(x, x)) + .window(TumblingWindows.of(BaseWindowedBolt.Count.of(10))) + .reduceByKey((x, y) -> x + y) + .reduceByKey((x, y) -> 0) + .print(); + StormTopology topology = streamBuilder.build(); + assertEquals(2, topology.get_bolts_size()); + } + + @Test + public void testMultiPartitionByKeyWithRepartition() { + TopologyContext mockContext = Mockito.mock(TopologyContext.class); + OutputCollector mockCollector = Mockito.mock(OutputCollector.class); + Map expected = new HashMap<>(); + expected.put(new GlobalStreamId("bolt2", "s3"), Grouping.fields(Collections.singletonList("key"))); + expected.put(new GlobalStreamId("bolt2", "s3__punctuation"), Grouping.all(new NullStruct())); + Stream stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new ValueMapper<>(0)); + stream.mapToPair(x -> Pair.of(x, x)) + .window(TumblingWindows.of(BaseWindowedBolt.Count.of(10))) + .reduceByKey((x, y) -> x + y) + .repartition(10) + .reduceByKey((x, y) -> 0) + .print(); + StormTopology topology = streamBuilder.build(); + assertEquals(3, topology.get_bolts_size()); + assertEquals(expected, topology.get_bolts().get("bolt3").get_common().get_inputs()); + + } + + @Test + public void testPartitionByKeySinglePartition() { + TopologyContext mockContext = Mockito.mock(TopologyContext.class); + OutputCollector mockCollector = Mockito.mock(OutputCollector.class); + Stream stream = streamBuilder.newStream(newSpout(Utils.DEFAULT_STREAM_ID), new ValueMapper<>(0)); + stream.mapToPair(x -> Pair.of(x, x)) + .reduceByKey((x, y) -> x + y) + .print(); + StormTopology topology = streamBuilder.build(); assertEquals(1, topology.get_bolts_size()); } diff --git a/storm-core/test/jvm/org/apache/storm/streams/WindowedProcessorBoltTest.java b/storm-core/test/jvm/org/apache/storm/streams/WindowedProcessorBoltTest.java index 7428e3f66ca..1010f6fe7dd 100644 --- a/storm-core/test/jvm/org/apache/storm/streams/WindowedProcessorBoltTest.java +++ b/storm-core/test/jvm/org/apache/storm/streams/WindowedProcessorBoltTest.java @@ -77,7 +77,7 @@ public void testEmit() throws Exception { Mockito.verify(mockOutputCollector, Mockito.times(2)).emit(os.capture(), values.capture()); assertEquals("outputstream", os.getAllValues().get(0)); assertEquals(new Values(3L), values.getAllValues().get(0)); - assertEquals("outputstream", os.getAllValues().get(1)); + assertEquals("outputstream__punctuation", os.getAllValues().get(1)); assertEquals(new Values(WindowNode.PUNCTUATION), values.getAllValues().get(1)); } diff --git a/storm-core/test/jvm/org/apache/storm/streams/processors/JoinProcessorTest.java b/storm-core/test/jvm/org/apache/storm/streams/processors/JoinProcessorTest.java new file mode 100644 index 00000000000..a8ace8ac8ca --- /dev/null +++ b/storm-core/test/jvm/org/apache/storm/streams/processors/JoinProcessorTest.java @@ -0,0 +1,108 @@ +package org.apache.storm.streams.processors; + +import org.apache.storm.streams.Pair; +import org.apache.storm.streams.operations.PairValueJoiner; +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Set; + +import static org.junit.Assert.*; + +public class JoinProcessorTest { + JoinProcessor, Integer, Integer> joinProcessor; + String leftStream = "left"; + String rightStream = "right"; + List>>> res = new ArrayList<>(); + + ProcessorContext context = new ProcessorContext() { + @Override + public void forward(T input) { + res.add((Pair>>)input); + } + + @Override + public void forward(T input, String stream) { + } + + @Override + public boolean isWindowed() { + return true; + } + + @Override + public Set getWindowedParentStreams() { + return null; + } + }; + + List> leftKeyValeus = Arrays.asList( + Pair.of(2, 4), + Pair.of(5, 25), + Pair.of(7, 49) + ); + + List> rightKeyValues = Arrays.asList( + Pair.of(1, 1), + Pair.of(2, 8), + Pair.of(5, 125), + Pair.of(6, 216) + ); + + @Test + public void testInnerJoin() throws Exception { + joinProcessor = new JoinProcessor<>(leftStream, rightStream, new PairValueJoiner<>()); + processValues(); + assertEquals(Pair.of(2, Pair.of(4, 8)), res.get(0)); + assertEquals(Pair.of(5, Pair.of(25, 125)), res.get(1)); + } + + @Test + public void testLeftOuterJoin() throws Exception { + joinProcessor = new JoinProcessor<>(leftStream, rightStream, new PairValueJoiner<>(), + JoinProcessor.JoinType.OUTER, JoinProcessor.JoinType.INNER); + processValues(); + assertEquals(Pair.of(2, Pair.of(4, 8)), res.get(0)); + assertEquals(Pair.of(5, Pair.of(25, 125)), res.get(1)); + assertEquals(Pair.of(7, Pair.of(49, null)), res.get(2)); + } + + @Test + public void testRightOuterJoin() throws Exception { + joinProcessor = new JoinProcessor<>(leftStream, rightStream, new PairValueJoiner<>(), + JoinProcessor.JoinType.INNER, JoinProcessor.JoinType.OUTER); + processValues(); + assertEquals(Pair.of(1, Pair.of(null, 1)), res.get(0)); + assertEquals(Pair.of(2, Pair.of(4, 8)), res.get(1)); + assertEquals(Pair.of(5, Pair.of(25, 125)), res.get(2)); + assertEquals(Pair.of(6, Pair.of(null, 216)), res.get(3)); + } + + @Test + public void testFullOuterJoin() throws Exception { + joinProcessor = new JoinProcessor<>(leftStream, rightStream, new PairValueJoiner<>(), + JoinProcessor.JoinType.OUTER, JoinProcessor.JoinType.OUTER); + processValues(); + assertEquals(Pair.of(1, Pair.of(null, 1)), res.get(0)); + assertEquals(Pair.of(2, Pair.of(4, 8)), res.get(1)); + assertEquals(Pair.of(5, Pair.of(25, 125)), res.get(2)); + assertEquals(Pair.of(6, Pair.of(null, 216)), res.get(3)); + assertEquals(Pair.of(7, Pair.of(49, null)), res.get(4)); + } + + private void processValues() { + res.clear(); + joinProcessor.init(context); + for (Pair kv : leftKeyValeus) { + joinProcessor.execute(kv, leftStream); + } + for (Pair kv : rightKeyValues) { + joinProcessor.execute(kv, rightStream); + } + joinProcessor.finish(); + } + +} \ No newline at end of file From dc19597c290ca0177cf9a06c81f13a60a3afedd2 Mon Sep 17 00:00:00 2001 From: Arun Mahadevan Date: Thu, 29 Dec 2016 15:16:34 +0530 Subject: [PATCH 03/29] [STORM-1961] Fields grouping for state query and refactored StreamBuilder --- .../starter/streams/StateQueryExample.java | 2 +- .../jvm/org/apache/storm/streams/Node.java | 2 +- .../org/apache/storm/streams/PairStream.java | 8 +- .../jvm/org/apache/storm/streams/Stream.java | 16 ++- .../apache/storm/streams/StreamBuilder.java | 113 ++++++++---------- 5 files changed, 63 insertions(+), 78 deletions(-) diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java index 2f0a4a3eec4..6d6a4b34cfd 100644 --- a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java @@ -53,7 +53,7 @@ public class StateQueryExample { public static void main(String[] args) throws Exception { StreamBuilder builder = new StreamBuilder(); - StreamState ss = builder.newStream(new TestWordSpout(), new ValueMapper(0)) + StreamState ss = builder.newStream(new TestWordSpout(), new ValueMapper(0), 2) .mapToPair(w -> Pair.of(w, 1)) .updateStateByKey(0L, (count, val) -> count + 1); diff --git a/storm-core/src/jvm/org/apache/storm/streams/Node.java b/storm-core/src/jvm/org/apache/storm/streams/Node.java index 3507f508227..d21dee97d05 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/Node.java +++ b/storm-core/src/jvm/org/apache/storm/streams/Node.java @@ -91,7 +91,7 @@ void setComponentId(String componentId) { this.componentId = componentId; } - Integer getParallelism() { + int getParallelism() { return parallelism; } diff --git a/storm-core/src/jvm/org/apache/storm/streams/PairStream.java b/storm-core/src/jvm/org/apache/storm/streams/PairStream.java index 964cdbaff53..69e6c37a1b6 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/PairStream.java +++ b/storm-core/src/jvm/org/apache/storm/streams/PairStream.java @@ -420,13 +420,7 @@ private boolean shouldPartitionByKey() { } private PairStream partitionBy(Fields fields) { - return partitionBy(fields, node.parallelism); - } - - private PairStream partitionBy(Fields fields, int parallelism) { - return new PairStream<>( - streamBuilder, - addNode(node, new PartitionNode(stream, node.getOutputFields(), GroupingInfo.fields(fields)), parallelism)); + return toPairStream(partitionBy(fields, node.parallelism)); } private PairStream toPairStream(Stream> stream) { diff --git a/storm-core/src/jvm/org/apache/storm/streams/Stream.java b/storm-core/src/jvm/org/apache/storm/streams/Stream.java index d553390a987..ef03ae31738 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/Stream.java +++ b/storm-core/src/jvm/org/apache/storm/streams/Stream.java @@ -370,9 +370,9 @@ public void to(IBasicBolt bolt, int parallelism) { * @return the result stream */ public PairStream stateQuery(StreamState streamState) { - // need all grouping for state query since the state is per-task - Node node = all().addProcessorNode(new StateQueryProcessor<>(streamState), KEY_VALUE); - return new PairStream<>(streamBuilder, node); + // need field grouping for state query so that the query is routed to the correct task + Node newNode = partitionBy(VALUE, node.getParallelism()).addProcessorNode(new StateQueryProcessor<>(streamState), KEY_VALUE); + return new PairStream<>(streamBuilder, newNode); } Node getNode() { @@ -435,12 +435,10 @@ private Stream global() { return new Stream<>(streamBuilder, partitionNode); } - private Stream all() { - if (node.getParallelism() == 1) { - return this; - } - Node partitionNode = addNode(new PartitionNode(stream, node.getOutputFields(), GroupingInfo.all())); - return new Stream<>(streamBuilder, partitionNode); + protected Stream partitionBy(Fields fields, int parallelism) { + return new Stream<>( + streamBuilder, + addNode(node, new PartitionNode(stream, node.getOutputFields(), GroupingInfo.fields(fields)), parallelism)); } private boolean shouldPartition() { diff --git a/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java b/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java index 7dff25d1afc..0bf02be4986 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java +++ b/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java @@ -17,10 +17,7 @@ */ package org.apache.storm.streams; -import com.google.common.base.Function; -import com.google.common.base.Predicate; import com.google.common.collect.ArrayListMultimap; -import com.google.common.collect.Collections2; import com.google.common.collect.HashBasedTable; import com.google.common.collect.Multimap; import com.google.common.collect.Table; @@ -28,7 +25,6 @@ import org.apache.storm.streams.operations.IdentityFunction; import org.apache.storm.streams.operations.mappers.PairValueMapper; import org.apache.storm.streams.operations.mappers.TupleValueMapper; -import org.apache.storm.streams.processors.JoinProcessor; import org.apache.storm.streams.processors.MapProcessor; import org.apache.storm.streams.processors.Processor; import org.apache.storm.streams.processors.StateQueryProcessor; @@ -48,6 +44,7 @@ import org.slf4j.LoggerFactory; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; @@ -56,6 +53,7 @@ import java.util.Map; import java.util.PriorityQueue; import java.util.Set; +import java.util.stream.Collectors; /** * A builder for constructing a {@link StormTopology} via storm streams api (DSL) @@ -261,9 +259,9 @@ private PriorityQueue queue() { p.put(SpoutNode.class, 0); p.put(UpdateStateByKeyProcessor.class, 1); p.put(ProcessorNode.class, 2); - p.put(StateQueryProcessor.class, 3); - p.put(PartitionNode.class, 4); - p.put(WindowNode.class, 5); + p.put(PartitionNode.class, 3); + p.put(WindowNode.class, 4); + p.put(StateQueryProcessor.class, 5); p.put(SinkNode.class, 6); } @Override @@ -384,30 +382,36 @@ private Set parentNodes(Node curNode) { return nodes; } + private Collection> parallelismGroups(List processorNodes) { + return processorNodes.stream().collect(Collectors.groupingBy(Node::getParallelism)).values(); + } + private void processCurGroup(TopologyBuilder topologyBuilder) { - if (curGroup.isEmpty()) { - return; + if (!curGroup.isEmpty()) { + parallelismGroups(curGroup).forEach(g -> doProcessCurGroup(topologyBuilder, g)); + curGroup.clear(); } + } + private void doProcessCurGroup(TopologyBuilder topologyBuilder, List group) { String boltId = UniqueIdGen.getInstance().getUniqueBoltId(); - for (ProcessorNode processorNode : curGroup) { + for (ProcessorNode processorNode : group) { processorNode.setComponentId(boltId); processorNode.setWindowedParentStreams(getWindowedParentStreams(processorNode)); } - final Set initialProcessors = initialProcessors(curGroup); + final Set initialProcessors = initialProcessors(group); Set> windowParams = getWindowParams(initialProcessors); if (windowParams.isEmpty()) { - if (hasStatefulProcessor(curGroup)) { - addStatefulBolt(topologyBuilder, boltId, initialProcessors); + if (hasStatefulProcessor(group)) { + addStatefulBolt(topologyBuilder, boltId, initialProcessors, group); } else { - addBolt(topologyBuilder, boltId, initialProcessors); + addBolt(topologyBuilder, boltId, initialProcessors, group); } } else if (windowParams.size() == 1) { - addWindowedBolt(topologyBuilder, boltId, initialProcessors, windowParams.iterator().next()); + addWindowedBolt(topologyBuilder, boltId, initialProcessors, windowParams.iterator().next(), group); } else { - throw new IllegalStateException("More than one window config for current group " + curGroup); + throw new IllegalStateException("More than one window config for current group " + group); } - curGroup.clear(); } private boolean hasStatefulProcessor(List processorNodes) { @@ -419,16 +423,11 @@ private boolean hasStatefulProcessor(List processorNodes) { return false; } - private int getParallelism() { - Set parallelisms = new HashSet<>(Collections2.transform(curGroup, new Function() { - @Override - public Integer apply(ProcessorNode input) { - return input.getParallelism(); - } - })); + private int getParallelism(List group) { + Set parallelisms = group.stream().map(Node::getParallelism).collect(Collectors.toSet()); if (parallelisms.size() > 1) { - throw new IllegalStateException("Current group does not have same parallelism " + curGroup); + throw new IllegalStateException("Current group does not have same parallelism " + group); } return parallelisms.isEmpty() ? 1 : parallelisms.iterator().next(); @@ -446,16 +445,7 @@ public Integer apply(ProcessorNode input) { } } - Set> windowParams = new HashSet<>(); - if (!windowNodes.isEmpty()) { - windowParams.addAll(new HashSet<>(Collections2.transform(windowNodes, new Function>() { - @Override - public Window apply(WindowNode input) { - return input.getWindowParams(); - } - }))); - } - return windowParams; + return windowNodes.stream().map(WindowNode::getWindowParams).collect(Collectors.toSet()); } private void addSpout(TopologyBuilder topologyBuilder, SpoutNode spout) { @@ -481,39 +471,41 @@ private void addSink(TopologyBuilder topologyBuilder, SinkNode sinkNode) { private StreamBolt addBolt(TopologyBuilder topologyBuilder, String boltId, - Set initialProcessors) { - ProcessorBolt bolt = new ProcessorBolt(boltId, graph, curGroup); - BoltDeclarer boltDeclarer = topologyBuilder.setBolt(boltId, bolt, getParallelism()); - bolt.setStreamToInitialProcessors(wireBolt(curGroup, boltDeclarer, initialProcessors)); + Set initialProcessors, + List group) { + ProcessorBolt bolt = new ProcessorBolt(boltId, graph, group); + BoltDeclarer boltDeclarer = topologyBuilder.setBolt(boltId, bolt, getParallelism(group)); + bolt.setStreamToInitialProcessors(wireBolt(group, boltDeclarer, initialProcessors)); streamBolts.put(bolt, boltDeclarer); return bolt; } private StreamBolt addStatefulBolt(TopologyBuilder topologyBuilder, String boltId, - Set initialProcessors) { - StateQueryProcessor stateQueryProcessor = getStateQueryProcessor(); + Set initialProcessors, + List group) { + StateQueryProcessor stateQueryProcessor = getStateQueryProcessor(group); StatefulProcessorBolt bolt; if (stateQueryProcessor == null) { - bolt = new StatefulProcessorBolt<>(boltId, graph, curGroup); - BoltDeclarer boltDeclarer = topologyBuilder.setBolt(boltId, bolt, getParallelism()); - bolt.setStreamToInitialProcessors(wireBolt(curGroup, boltDeclarer, initialProcessors)); + bolt = new StatefulProcessorBolt<>(boltId, graph, group); + BoltDeclarer boltDeclarer = topologyBuilder.setBolt(boltId, bolt, getParallelism(group)); + bolt.setStreamToInitialProcessors(wireBolt(group, boltDeclarer, initialProcessors)); streamBolts.put(bolt, boltDeclarer); } else { // state query is added to the existing stateful bolt ProcessorNode updateStateNode = stateQueryProcessor.getStreamState().getUpdateStateNode(); bolt = findStatefulProcessorBolt(updateStateNode); - for (ProcessorNode node : curGroup) { + for (ProcessorNode node : group) { node.setComponentId(bolt.getId()); } - bolt.addNodes(curGroup); + bolt.addNodes(group); bolt.addStreamToInitialProcessors(wireBolt(bolt.getNodes(), streamBolts.get(bolt), initialProcessors)); } return bolt; } - private StateQueryProcessor getStateQueryProcessor() { - for (ProcessorNode node : curGroup) { + private StateQueryProcessor getStateQueryProcessor(List group) { + for (ProcessorNode node : group) { if (node.getProcessor() instanceof StateQueryProcessor) { return (StateQueryProcessor) node.getProcessor(); } @@ -524,10 +516,11 @@ private StreamBolt addStatefulBolt(TopologyBuilder topologyBuilder, private StreamBolt addWindowedBolt(TopologyBuilder topologyBuilder, String boltId, Set initialProcessors, - Window windowParam) { - WindowedProcessorBolt bolt = new WindowedProcessorBolt(boltId, graph, curGroup, windowParam); - BoltDeclarer boltDeclarer = topologyBuilder.setBolt(boltId, bolt, getParallelism()); - bolt.setStreamToInitialProcessors(wireBolt(curGroup, boltDeclarer, initialProcessors)); + Window windowParam, + List group) { + WindowedProcessorBolt bolt = new WindowedProcessorBolt(boltId, graph, group, windowParam); + BoltDeclarer boltDeclarer = topologyBuilder.setBolt(boltId, bolt, getParallelism(group)); + bolt.setStreamToInitialProcessors(wireBolt(group, boltDeclarer, initialProcessors)); streamBolts.put(bolt, boltDeclarer); return bolt; } @@ -554,17 +547,17 @@ private Set getWindowedParentStreams(ProcessorNode processorNode) { return res; } - private Multimap wireBolt(List curGroup, + private Multimap wireBolt(List group, BoltDeclarer boltDeclarer, Set initialProcessors) { - LOG.debug("Wiring bolt with boltDeclarer {}, curGroup {}, initialProcessors {}, nodeGroupingInfo {}", - boltDeclarer, curGroup, initialProcessors, nodeGroupingInfo); + LOG.debug("Wiring bolt with boltDeclarer {}, group {}, initialProcessors {}, nodeGroupingInfo {}", + boltDeclarer, group, initialProcessors, nodeGroupingInfo); Multimap streamToInitialProcessor = ArrayListMultimap.create(); - Set curSet = new HashSet<>(curGroup); + Set curSet = new HashSet<>(group); for (ProcessorNode curNode : initialProcessors) { for (Node parent : parentNodes(curNode)) { if (curSet.contains(parent)) { - LOG.debug("Parent {} of curNode {} is in curGroup {}", parent, curNode, curGroup); + LOG.debug("Parent {} of curNode {} is in group {}", parent, curNode, group); } else { for (String stream : curNode.getParentStreams(parent)) { declareGrouping(boltDeclarer, parent, stream, nodeGroupingInfo.get(parent, stream)); @@ -592,10 +585,10 @@ private void declareGrouping(BoltDeclarer boltDeclarer, Node parent, String stre } } - private Set initialProcessors(List curGroup) { + private Set initialProcessors(List group) { Set nodes = new HashSet<>(); - Set curSet = new HashSet<>(curGroup); - for (ProcessorNode node : curGroup) { + Set curSet = new HashSet<>(group); + for (ProcessorNode node : group) { for (Node parent : parentNodes(node)) { if (!(parent instanceof ProcessorNode) || !curSet.contains(parent)) { nodes.add(node); From 0aa0e8579723a76f53fe7e0ae71dfd1722465c0e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stig=20Rohde=20D=C3=B8ssing?= Date: Tue, 2 Aug 2016 19:17:02 +0200 Subject: [PATCH 04/29] STORM-2014: Make KafkaSpout delegate maxRetry check to RetryService --- .../apache/storm/kafka/spout/KafkaSpout.java | 13 ++++---- .../storm/kafka/spout/KafkaSpoutConfig.java | 23 +------------- .../KafkaSpoutRetryExponentialBackoff.java | 30 +++++++++++-------- .../kafka/spout/KafkaSpoutRetryService.java | 12 ++++++-- 4 files changed, 34 insertions(+), 44 deletions(-) diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java index d405c4da874..60ca0b985e2 100644 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java @@ -72,7 +72,6 @@ public class KafkaSpout extends BaseRichSpout { // Bookkeeping - private transient int maxRetries; // Max number of times a tuple is retried private transient FirstPollOffsetStrategy firstPollOffsetStrategy; // Strategy to determine the fetch offset of the first realized by the spout upon activation private transient KafkaSpoutRetryService retryService; // Class that has the logic to handle tuple failure private transient Timer commitTimer; // timer == null for auto commit mode @@ -105,7 +104,6 @@ public void open(Map conf, TopologyContext context, SpoutOutputCollector collect // Spout internals this.collector = collector; - maxRetries = kafkaSpoutConfig.getMaxTupleRetries(); numUncommittedOffsets = 0; // Offset management @@ -381,11 +379,9 @@ public void fail(Object messageId) { LOG.debug("Received fail for tuple this spout is no longer tracking. Partitions may have been reassigned. Ignoring message [{}]", msgId); return; } - if (msgId.numFails() < maxRetries) { - emitted.remove(msgId); - msgId.incrementNumFails(); - retryService.schedule(msgId); - } else { // limit to max number of retries + emitted.remove(msgId); + msgId.incrementNumFails(); + if (!retryService.schedule(msgId)) { LOG.debug("Reached maximum number of retries. Message [{}] being marked as acked.", msgId); ack(msgId); } @@ -523,6 +519,9 @@ public void add(KafkaSpoutMessageId msgId) { // O(Log N) } /** + * An offset is only committed when all records with lower offset have + * been acked. This guarantees that all offsets smaller than the + * committedOffset have been delivered. * @return the next OffsetAndMetadata to commit, or null if no offset is ready to commit. */ public OffsetAndMetadata findNextCommitOffset() { diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java index 8aa525bdc77..2818c9a48f8 100644 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java @@ -74,7 +74,6 @@ public enum FirstPollOffsetStrategy { // Kafka spout configuration private final long offsetCommitPeriodMs; - private final int maxRetries; private final int maxUncommittedOffsets; private final FirstPollOffsetStrategy firstPollOffsetStrategy; private final KafkaSpoutStreams kafkaSpoutStreams; @@ -87,7 +86,6 @@ private KafkaSpoutConfig(Builder builder) { this.valueDeserializer = builder.valueDeserializer; this.pollTimeoutMs = builder.pollTimeoutMs; this.offsetCommitPeriodMs = builder.offsetCommitPeriodMs; - this.maxRetries = builder.maxRetries; this.firstPollOffsetStrategy = builder.firstPollOffsetStrategy; this.kafkaSpoutStreams = builder.kafkaSpoutStreams; this.maxUncommittedOffsets = builder.maxUncommittedOffsets; @@ -109,7 +107,6 @@ public static class Builder { private SerializableDeserializer valueDeserializer; private long pollTimeoutMs = DEFAULT_POLL_TIMEOUT_MS; private long offsetCommitPeriodMs = DEFAULT_OFFSET_COMMIT_PERIOD_MS; - private int maxRetries = DEFAULT_MAX_RETRIES; private FirstPollOffsetStrategy firstPollOffsetStrategy = FirstPollOffsetStrategy.UNCOMMITTED_EARLIEST; private final KafkaSpoutStreams kafkaSpoutStreams; private int maxUncommittedOffsets = DEFAULT_MAX_UNCOMMITTED_OFFSETS; @@ -194,20 +191,7 @@ public Builder setOffsetCommitPeriodMs(long offsetCommitPeriodMs) { this.offsetCommitPeriodMs = offsetCommitPeriodMs; return this; } - - /** - * Defines the max number of retrials in case of tuple failure. The default is to retry forever, which means that - * no new records are committed until the previous polled records have been acked. This guarantees at once delivery of - * all the previously polled records. - * By specifying a finite value for maxRetries, the user decides to sacrifice guarantee of delivery for the previous - * polled records in favor of processing more records. - * @param maxRetries max number of retrials - */ - public Builder setMaxRetries(int maxRetries) { - this.maxRetries = maxRetries; - return this; - } - + /** * Defines the max number of polled offsets (records) that can be pending commit, before another poll can take place. * Once this limit is reached, no more offsets (records) can be polled until the next successful commit(s) sets the number @@ -283,10 +267,6 @@ public Pattern getTopicWildcardPattern() { null; } - public int getMaxTupleRetries() { - return maxRetries; - } - public FirstPollOffsetStrategy getFirstPollOffsetStrategy() { return firstPollOffsetStrategy; } @@ -315,7 +295,6 @@ public String toString() { ", valueDeserializer=" + valueDeserializer + ", pollTimeoutMs=" + pollTimeoutMs + ", offsetCommitPeriodMs=" + offsetCommitPeriodMs + - ", maxRetries=" + maxRetries + ", maxUncommittedOffsets=" + maxUncommittedOffsets + ", firstPollOffsetStrategy=" + firstPollOffsetStrategy + ", kafkaSpoutStreams=" + kafkaSpoutStreams + diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutRetryExponentialBackoff.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutRetryExponentialBackoff.java index f59367da634..2c8d7e4d19f 100644 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutRetryExponentialBackoff.java +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutRetryExponentialBackoff.java @@ -40,25 +40,26 @@ public class KafkaSpoutRetryExponentialBackoff implements KafkaSpoutRetryService private static final Logger LOG = LoggerFactory.getLogger(KafkaSpoutRetryExponentialBackoff.class); private static final RetryEntryTimeStampComparator RETRY_ENTRY_TIME_STAMP_COMPARATOR = new RetryEntryTimeStampComparator(); - private TimeInterval initialDelay; - private TimeInterval delayPeriod; - private TimeInterval maxDelay; - private int maxRetries; + private final TimeInterval initialDelay; + private final TimeInterval delayPeriod; + private final TimeInterval maxDelay; + private final int maxRetries; - private Set retrySchedules = new TreeSet<>(RETRY_ENTRY_TIME_STAMP_COMPARATOR); - private Set toRetryMsgs = new HashSet<>(); // Convenience data structure to speedup lookups + private final Set retrySchedules = new TreeSet<>(RETRY_ENTRY_TIME_STAMP_COMPARATOR); + private final Set toRetryMsgs = new HashSet<>(); // Convenience data structure to speedup lookups /** * Comparator ordering by timestamp */ private static class RetryEntryTimeStampComparator implements Serializable, Comparator { + @Override public int compare(RetrySchedule entry1, RetrySchedule entry2) { return Long.valueOf(entry1.nextRetryTimeNanos()).compareTo(entry2.nextRetryTimeNanos()); } } private class RetrySchedule { - private KafkaSpoutMessageId msgId; + private final KafkaSpoutMessageId msgId; private long nextRetryTimeNanos; public RetrySchedule(KafkaSpoutMessageId msgId, long nextRetryTime) { @@ -94,9 +95,9 @@ public long nextRetryTimeNanos() { } public static class TimeInterval implements Serializable { - private long lengthNanos; - private long length; - private TimeUnit timeUnit; + private final long lengthNanos; + private final long length; + private final TimeUnit timeUnit; /** * @param length length of the time interval in the units specified by {@link TimeUnit} @@ -144,7 +145,10 @@ public String toString() { /** * The time stamp of the next retry is scheduled according to the exponential backoff formula ( geometric progression): * nextRetry = failCount == 1 ? currentTime + initialDelay : currentTime + delayPeriod^(failCount-1) where failCount = 1, 2, 3, ... - * nextRetry = Min(nextRetry, currentTime + maxDelay) + * nextRetry = Min(nextRetry, currentTime + maxDelay). + * + * By specifying a value for maxRetries lower than Integer.MAX_VALUE, the user decides to sacrifice guarantee of delivery for the previous + * polled records in favor of processing more records. * * @param initialDelay initial delay of the first retry * @param delayPeriod the time interval that is the ratio of the exponential backoff formula (geometric progression) @@ -239,9 +243,10 @@ public boolean retainAll(Collection topicPartitions) { } @Override - public void schedule(KafkaSpoutMessageId msgId) { + public boolean schedule(KafkaSpoutMessageId msgId) { if (msgId.numFails() > maxRetries) { LOG.debug("Not scheduling [{}] because reached maximum number of retries [{}].", msgId, maxRetries); + return false; } else { if (toRetryMsgs.contains(msgId)) { for (Iterator iterator = retrySchedules.iterator(); iterator.hasNext(); ) { @@ -257,6 +262,7 @@ public void schedule(KafkaSpoutMessageId msgId) { toRetryMsgs.add(msgId); LOG.debug("Scheduled. {}", retrySchedule); LOG.trace("Current state {}", retrySchedules); + return true; } } diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutRetryService.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutRetryService.java index 5aab1678de1..bf17a5ae639 100644 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutRetryService.java +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutRetryService.java @@ -29,14 +29,18 @@ */ public interface KafkaSpoutRetryService extends Serializable { /** - * Schedules this {@link KafkaSpoutMessageId} if not yet scheduled, or updates retry time if it has already been scheduled. + * Schedules this {@link KafkaSpoutMessageId} if not yet scheduled, or + * updates retry time if it has already been scheduled. It may also indicate + * that the message should not be retried, in which case the message will not be scheduled. * @param msgId message to schedule for retrial + * @return true if the message will be retried, false otherwise */ - void schedule(KafkaSpoutMessageId msgId); + boolean schedule(KafkaSpoutMessageId msgId); /** * Removes a message from the list of messages scheduled for retrial * @param msgId message to remove from retrial + * @return true if the message was scheduled for retrial, false otherwise */ boolean remove(KafkaSpoutMessageId msgId); @@ -56,8 +60,9 @@ public interface KafkaSpoutRetryService extends Serializable { Set retriableTopicPartitions(); /** - * Checks if a specific failed {@link KafkaSpoutMessageId} is is ready to be retried, + * Checks if a specific failed {@link KafkaSpoutMessageId} is ready to be retried, * i.e is scheduled and has retry time that is less than current time. + * @param msgId message to check for readiness * @return true if message is ready to be retried, false otherwise */ boolean isReady(KafkaSpoutMessageId msgId); @@ -65,6 +70,7 @@ public interface KafkaSpoutRetryService extends Serializable { /** * Checks if a specific failed {@link KafkaSpoutMessageId} is scheduled to be retried. * The message may or may not be ready to be retried yet. + * @param msgId message to check for scheduling status * @return true if the message is scheduled to be retried, regardless of being or not ready to be retried. * Returns false is this message is not scheduled for retrial */ From a07039fc07f6f2aa5d3baeaf5107359218d84b5c Mon Sep 17 00:00:00 2001 From: Arun Mahadevan Date: Fri, 20 Jan 2017 10:33:09 +0530 Subject: [PATCH 05/29] [STORM-1961] Added interface stability annotation 1. Added interface stability annotation (borrowed from hadoop) 2. Marked relevant stream api classes 'Unstable' --- .../storm/annotation/InterfaceStability.java | 54 +++++++++++++++++++ .../org/apache/storm/streams/PairStream.java | 2 + .../jvm/org/apache/storm/streams/Stream.java | 2 + .../apache/storm/streams/StreamBuilder.java | 2 + .../storm/streams/processors/Processor.java | 3 ++ .../streams/processors/ProcessorContext.java | 3 ++ 6 files changed, 66 insertions(+) create mode 100644 storm-core/src/jvm/org/apache/storm/annotation/InterfaceStability.java diff --git a/storm-core/src/jvm/org/apache/storm/annotation/InterfaceStability.java b/storm-core/src/jvm/org/apache/storm/annotation/InterfaceStability.java new file mode 100644 index 00000000000..d05ae75ad55 --- /dev/null +++ b/storm-core/src/jvm/org/apache/storm/annotation/InterfaceStability.java @@ -0,0 +1,54 @@ +/** + * 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.storm.annotation; + +import java.lang.annotation.Documented; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; + +/** + * Annotation to inform users of how much to rely on a particular package, + * class or method not changing over time. + * + */ +@InterfaceStability.Evolving +public class InterfaceStability { + /** + * Can evolve while retaining compatibility for minor release boundaries.; + * can break compatibility only at major release (ie. at m.0). + */ + @Documented + @Retention(RetentionPolicy.RUNTIME) + public @interface Stable {}; + + /** + * Evolving, but can break compatibility at minor release (i.e. m.x) + */ + @Documented + @Retention(RetentionPolicy.RUNTIME) + public @interface Evolving {}; + + /** + * No guarantee is provided as to reliability or stability across any + * level of release granularity. + */ + @Documented + @Retention(RetentionPolicy.RUNTIME) + public @interface Unstable {}; +} + diff --git a/storm-core/src/jvm/org/apache/storm/streams/PairStream.java b/storm-core/src/jvm/org/apache/storm/streams/PairStream.java index 69e6c37a1b6..3c08a05431d 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/PairStream.java +++ b/storm-core/src/jvm/org/apache/storm/streams/PairStream.java @@ -18,6 +18,7 @@ package org.apache.storm.streams; import org.apache.storm.Config; +import org.apache.storm.annotation.InterfaceStability; import org.apache.storm.streams.operations.BiFunction; import org.apache.storm.streams.operations.CombinerAggregator; import org.apache.storm.streams.operations.Consumer; @@ -50,6 +51,7 @@ * @param the key type * @param the value type */ +@InterfaceStability.Unstable public class PairStream extends Stream> { PairStream(StreamBuilder topology, Node node) { diff --git a/storm-core/src/jvm/org/apache/storm/streams/Stream.java b/storm-core/src/jvm/org/apache/storm/streams/Stream.java index ef03ae31738..087e7609649 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/Stream.java +++ b/storm-core/src/jvm/org/apache/storm/streams/Stream.java @@ -17,6 +17,7 @@ */ package org.apache.storm.streams; +import org.apache.storm.annotation.InterfaceStability; import org.apache.storm.streams.operations.BiFunction; import org.apache.storm.streams.operations.CombinerAggregator; import org.apache.storm.streams.operations.Consumer; @@ -58,6 +59,7 @@ * * @param the type of the value */ +@InterfaceStability.Unstable public class Stream { private static final Logger LOG = LoggerFactory.getLogger(Stream.class); diff --git a/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java b/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java index 0bf02be4986..8b6c57db572 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java +++ b/storm-core/src/jvm/org/apache/storm/streams/StreamBuilder.java @@ -21,6 +21,7 @@ import com.google.common.collect.HashBasedTable; import com.google.common.collect.Multimap; import com.google.common.collect.Table; +import org.apache.storm.annotation.InterfaceStability; import org.apache.storm.generated.StormTopology; import org.apache.storm.streams.operations.IdentityFunction; import org.apache.storm.streams.operations.mappers.PairValueMapper; @@ -58,6 +59,7 @@ /** * A builder for constructing a {@link StormTopology} via storm streams api (DSL) */ +@InterfaceStability.Unstable public class StreamBuilder { private static final Logger LOG = LoggerFactory.getLogger(StreamBuilder.class); private final DefaultDirectedGraph graph; diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/Processor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/Processor.java index 5551682739c..3b4518d27e3 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/processors/Processor.java +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/Processor.java @@ -17,6 +17,8 @@ */ package org.apache.storm.streams.processors; +import org.apache.storm.annotation.InterfaceStability; + import java.io.Serializable; /** @@ -24,6 +26,7 @@ * * @param the type of the input that is processed */ +@InterfaceStability.Unstable public interface Processor extends Serializable { /** * Initializes the processor. This is typically invoked from the underlying diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/ProcessorContext.java b/storm-core/src/jvm/org/apache/storm/streams/processors/ProcessorContext.java index 2a10a224eb8..a69a0a9fabb 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/processors/ProcessorContext.java +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/ProcessorContext.java @@ -17,12 +17,15 @@ */ package org.apache.storm.streams.processors; +import org.apache.storm.annotation.InterfaceStability; + import java.io.Serializable; import java.util.Set; /** * Context information passed to the {@link Processor}. */ +@InterfaceStability.Unstable public interface ProcessorContext extends Serializable { /** * Forwards the input to all downstream processors. From 37ee12ccd9a0465dc6cd6d2f9fb59929a798b6b5 Mon Sep 17 00:00:00 2001 From: Arun Mahadevan Date: Tue, 24 Jan 2017 17:35:32 +0530 Subject: [PATCH 06/29] [STORM-1961] Addressed review comments and a few other refactoring --- .../starter/streams/AggregateExample.java | 17 ++++++----- .../storm/starter/streams/BranchExample.java | 9 +++--- .../streams/GroupByKeyAndWindowExample.java | 23 +++++++++------ .../storm/starter/streams/JoinExample.java | 9 +++--- .../starter/streams/StateQueryExample.java | 16 +++++++---- .../starter/streams/StatefulWordCount.java | 15 +++++----- .../starter/streams/TypedTupleExample.java | 9 +++--- .../starter/streams/WindowedWordCount.java | 9 +++--- .../starter/streams/WordCountToBolt.java | 9 +++--- .../jvm/org/apache/storm/streams/Pair.java | 28 +++++++++++-------- .../org/apache/storm/streams/PairStream.java | 3 +- .../jvm/org/apache/storm/streams/Stream.java | 5 ++-- .../aggregators/{Sum.java => LongSum.java} | 2 +- .../processors/AggregateByKeyProcessor.java | 5 ++-- .../processors/AggregateProcessor.java | 4 +-- .../streams/processors/BaseProcessor.java | 8 ++++-- .../MergeAggregateByKeyProcessor.java | 2 +- .../processors/MergeAggregateProcessor.java | 2 +- .../storm/streams/processors/Processor.java | 2 +- .../streams/processors/ProcessorContext.java | 4 +-- .../processors/ReduceByKeyProcessor.java | 6 ++-- .../streams/processors/ReduceProcessor.java | 2 +- .../storm/streams/ProcessorBoltTest.java | 4 +-- 23 files changed, 105 insertions(+), 88 deletions(-) rename storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/{Sum.java => LongSum.java} (94%) diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java index 91dfadb2df1..74676348818 100644 --- a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/AggregateExample.java @@ -60,11 +60,10 @@ public static void main(String[] args) throws Exception { config.setNumWorkers(1); StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build()); } else { - LocalCluster cluster = new LocalCluster(); - cluster.submitTopology("test", config, builder.build()); - Utils.sleep(60000); - cluster.killTopology("test"); - cluster.shutdown(); + try (LocalCluster cluster = new LocalCluster(); + LocalCluster.LocalTopology topo = cluster.submitTopology("test", config, builder.build())) { + Utils.sleep(60_000); + } } } @@ -76,21 +75,21 @@ public Pair init() { @Override public Pair apply(Pair sumAndCount, Integer value) { - return Pair.of(sumAndCount.getFirst() + value, sumAndCount.getSecond() + 1); + return Pair.of(sumAndCount._1 + value, sumAndCount._2 + 1); } @Override public Pair merge(Pair sumAndCount1, Pair sumAndCount2) { System.out.println("Merge " + sumAndCount1 + " and " + sumAndCount2); return Pair.of( - sumAndCount1.getFirst() + sumAndCount2.getFirst(), - sumAndCount1.getSecond() + sumAndCount2.getSecond() + sumAndCount1._1 + sumAndCount2._1, + sumAndCount1._2 + sumAndCount2._2 ); } @Override public Double result(Pair sumAndCount) { - return (double) sumAndCount.getFirst()/sumAndCount.getSecond(); + return (double) sumAndCount._1/sumAndCount._2; } } } diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/BranchExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/BranchExample.java index f5400a5cbbc..027b432ff84 100644 --- a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/BranchExample.java +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/BranchExample.java @@ -61,11 +61,10 @@ public static void main(String[] args) throws Exception { config.setNumWorkers(1); StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build()); } else { - LocalCluster cluster = new LocalCluster(); - cluster.submitTopology("test", config, builder.build()); - Utils.sleep(60000); - cluster.killTopology("test"); - cluster.shutdown(); + try (LocalCluster cluster = new LocalCluster(); + LocalCluster.LocalTopology topo = cluster.submitTopology("test", config, builder.build())) { + Utils.sleep(60_000); + } } } diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/GroupByKeyAndWindowExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/GroupByKeyAndWindowExample.java index 6b505bdc95e..dd7e97f3408 100644 --- a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/GroupByKeyAndWindowExample.java +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/GroupByKeyAndWindowExample.java @@ -52,6 +52,9 @@ public static void main(String[] args) throws Exception { /* * The elements having the same key within the window will be grouped * together and the corresponding values will be merged. + * + * The result is a PairStream> with + * 'stock symbol' as the key and 'stock prices' for that symbol within the window as the value. */ .groupByKeyAndWindow(SlidingWindows.of(Count.of(6), Count.of(3))) .print(); @@ -61,8 +64,11 @@ public static void main(String[] args) throws Exception { /* * The elements having the same key within the window will be grouped * together and their values will be reduced using the given reduce function. + * + * Here the result is a PairStream with + * 'stock symbol' as the key and the maximum price for that symbol within the window as the value. */ - .reduceByKeyAndWindow((x, y) -> (x + y) / 2.0, SlidingWindows.of(Count.of(6), Count.of(3))) + .reduceByKeyAndWindow((x, y) -> x > y ? x : y, SlidingWindows.of(Count.of(6), Count.of(3))) .print(); Config config = new Config(); @@ -70,19 +76,18 @@ public static void main(String[] args) throws Exception { config.setNumWorkers(1); StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build()); } else { - LocalCluster cluster = new LocalCluster(); - config.put(Config.TOPOLOGY_STATE_PROVIDER, "org.apache.storm.redis.state.RedisKeyValueStateProvider"); - cluster.submitTopology("test", config, builder.build()); - Utils.sleep(60000); - cluster.killTopology("test"); - cluster.shutdown(); + try (LocalCluster cluster = new LocalCluster(); + LocalCluster.LocalTopology topo = cluster.submitTopology("test", config, builder.build())) { + Utils.sleep(60_000); + } } } private static class StockQuotes extends BaseRichSpout { private final List> values = Arrays.asList( Arrays.asList(new Values("AAPL", 100.0), new Values("GOOG", 780.0), new Values("FB", 125.0)), - Arrays.asList(new Values("AAPL", 105.0), new Values("GOOG", 790.0), new Values("FB", 130.0)) + Arrays.asList(new Values("AAPL", 105.0), new Values("GOOG", 790.0), new Values("FB", 130.0)), + Arrays.asList(new Values("AAPL", 102.0), new Values("GOOG", 788.0), new Values("FB", 128.0)) ); private SpoutOutputCollector collector; private int index = 0; @@ -103,7 +108,7 @@ public void nextTuple() { @Override public void declareOutputFields(OutputFieldsDeclarer declarer) { - declarer.declare(new Fields("key", "val")); + declarer.declare(new Fields("symbol", "price")); } } } diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/JoinExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/JoinExample.java index 0b15615f2bb..4aa62538234 100644 --- a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/JoinExample.java +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/JoinExample.java @@ -71,11 +71,10 @@ public static void main(String[] args) throws Exception { config.setNumWorkers(1); StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build()); } else { - LocalCluster cluster = new LocalCluster(); - cluster.submitTopology("test", config, builder.build()); - Utils.sleep(60000); - cluster.killTopology("test"); - cluster.shutdown(); + try (LocalCluster cluster = new LocalCluster(); + LocalCluster.LocalTopology topo = cluster.submitTopology("test", config, builder.build())) { + Utils.sleep(60_000); + } } } diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java index 6d6a4b34cfd..ab6cac353fe 100644 --- a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StateQueryExample.java @@ -54,7 +54,14 @@ public class StateQueryExample { public static void main(String[] args) throws Exception { StreamBuilder builder = new StreamBuilder(); StreamState ss = builder.newStream(new TestWordSpout(), new ValueMapper(0), 2) + /* + * Transform the stream of words to a stream of (word, 1) pairs + */ .mapToPair(w -> Pair.of(w, 1)) + /* + * Update the count in the state. Here the first argument 0L is the initial value for the count and + * the second argument is a function that increments the count for each value received. + */ .updateStateByKey(0L, (count, val) -> count + 1); /* @@ -77,11 +84,10 @@ public static void main(String[] args) throws Exception { config.setNumWorkers(1); StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build()); } else { - LocalCluster cluster = new LocalCluster(); - cluster.submitTopology("test", config, builder.build()); - Utils.sleep(60000); - cluster.killTopology("test"); - cluster.shutdown(); + try (LocalCluster cluster = new LocalCluster(); + LocalCluster.LocalTopology topo = cluster.submitTopology("test", config, builder.build())) { + Utils.sleep(60_000); + } } } diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StatefulWordCount.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StatefulWordCount.java index ce7470dfab1..ddd318a61b4 100644 --- a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StatefulWordCount.java +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/StatefulWordCount.java @@ -60,9 +60,11 @@ public static void main(String[] args) throws Exception { */ .countByKey() /* - * update the word counts in the state + * update the word counts in the state. + * Here the first argument 0L is the initial value for the state + * and the second argument is a function that adds the count to the current value in the state. */ - .updateStateByKey(0L, (x, y) -> x + y) + .updateStateByKey(0L, (state, count) -> state + count) /* * convert the state back to a stream and print the results */ @@ -77,11 +79,10 @@ public static void main(String[] args) throws Exception { config.setNumWorkers(1); StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build()); } else { - LocalCluster cluster = new LocalCluster(); - cluster.submitTopology("test", config, builder.build()); - Utils.sleep(60000); - cluster.killTopology("test"); - cluster.shutdown(); + try (LocalCluster cluster = new LocalCluster(); + LocalCluster.LocalTopology topo = cluster.submitTopology("test", config, builder.build())) { + Utils.sleep(60_000); + } } } } diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/TypedTupleExample.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/TypedTupleExample.java index 193ad661191..11e89bf9379 100644 --- a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/TypedTupleExample.java +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/TypedTupleExample.java @@ -54,11 +54,10 @@ public static void main(String[] args) throws Exception { config.setNumWorkers(1); StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build()); } else { - LocalCluster cluster = new LocalCluster(); - cluster.submitTopology("test", config, builder.build()); - Utils.sleep(60000); - cluster.killTopology("test"); - cluster.shutdown(); + try (LocalCluster cluster = new LocalCluster(); + LocalCluster.LocalTopology topo = cluster.submitTopology("test", config, builder.build())) { + Utils.sleep(60_000); + } } } } diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WindowedWordCount.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WindowedWordCount.java index 0765a741dbf..0f30b7cdb41 100644 --- a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WindowedWordCount.java +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WindowedWordCount.java @@ -70,11 +70,10 @@ public static void main(String[] args) throws Exception { config.setNumWorkers(1); StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build()); } else { - LocalCluster cluster = new LocalCluster(); - cluster.submitTopology("test", config, builder.build()); - Utils.sleep(60000); - cluster.killTopology("test"); - cluster.shutdown(); + try (LocalCluster cluster = new LocalCluster(); + LocalCluster.LocalTopology topo = cluster.submitTopology("test", config, builder.build())) { + Utils.sleep(60_000); + } } } } diff --git a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WordCountToBolt.java b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WordCountToBolt.java index dd7923a8bdb..1c0aae164a8 100644 --- a/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WordCountToBolt.java +++ b/examples/storm-starter/src/jvm/org/apache/storm/starter/streams/WordCountToBolt.java @@ -71,11 +71,10 @@ public static void main(String[] args) throws Exception { config.setNumWorkers(1); StormSubmitter.submitTopologyWithProgressBar(args[0], config, builder.build()); } else { - LocalCluster cluster = new LocalCluster(); - cluster.submitTopology("test", config, builder.build()); - Utils.sleep(60000); - cluster.killTopology("test"); - cluster.shutdown(); + try (LocalCluster cluster = new LocalCluster(); + LocalCluster.LocalTopology topo = cluster.submitTopology("test", config, builder.build())) { + Utils.sleep(60_000); + } } } diff --git a/storm-core/src/jvm/org/apache/storm/streams/Pair.java b/storm-core/src/jvm/org/apache/storm/streams/Pair.java index e5eb792ecd8..26d53b75ee4 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/Pair.java +++ b/storm-core/src/jvm/org/apache/storm/streams/Pair.java @@ -26,8 +26,14 @@ * @param the type of the second value */ public final class Pair implements Serializable { - private final T1 first; - private final T2 second; + /** + * The first value + */ + public final T1 _1; + /** + * The second value + */ + public final T2 _2; /** * Constructs a new pair of values @@ -36,8 +42,8 @@ public final class Pair implements Serializable { * @param second the second value */ private Pair(T1 first, T2 second) { - this.first = first; - this.second = second; + _1 = first; + _2 = second; } /** @@ -46,7 +52,7 @@ private Pair(T1 first, T2 second) { * @return the first value */ public T1 getFirst() { - return first; + return _1; } /** @@ -55,7 +61,7 @@ public T1 getFirst() { * @return the second value */ public T2 getSecond() { - return second; + return _2; } /** @@ -78,20 +84,20 @@ public boolean equals(Object o) { Pair pair = (Pair) o; - if (first != null ? !first.equals(pair.first) : pair.first != null) return false; - return second != null ? second.equals(pair.second) : pair.second == null; + if (_1 != null ? !_1.equals(pair._1) : pair._1 != null) return false; + return _2 != null ? _2.equals(pair._2) : pair._2 == null; } @Override public int hashCode() { - int result = first != null ? first.hashCode() : 0; - result = 31 * result + (second != null ? second.hashCode() : 0); + int result = _1 != null ? _1.hashCode() : 0; + result = 31 * result + (_2 != null ? _2.hashCode() : 0); return result; } @Override public String toString() { - return "(" + first + ", " + second + ')'; + return "(" + _1 + ", " + _2 + ')'; } } diff --git a/storm-core/src/jvm/org/apache/storm/streams/PairStream.java b/storm-core/src/jvm/org/apache/storm/streams/PairStream.java index 3c08a05431d..da332a5abd1 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/PairStream.java +++ b/storm-core/src/jvm/org/apache/storm/streams/PairStream.java @@ -376,7 +376,8 @@ public StreamState updateStateByKey(R initialValue, * @return the {@link StreamState} which can be used to query the state */ public StreamState updateStateByKey(StateUpdater stateUpdater) { - return partitionByKey().updateStateByKeyPartition(stateUpdater); + // repartition so that state query fields grouping works correctly. this can be optimized further + return partitionBy(KEY).updateStateByKeyPartition(stateUpdater); } private StreamState updateStateByKeyPartition(StateUpdater stateUpdater) { diff --git a/storm-core/src/jvm/org/apache/storm/streams/Stream.java b/storm-core/src/jvm/org/apache/storm/streams/Stream.java index 087e7609649..272c954b561 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/Stream.java +++ b/storm-core/src/jvm/org/apache/storm/streams/Stream.java @@ -278,8 +278,9 @@ public Stream repartition(int parallelism) { * Returns an array of streams by splitting the given stream into multiple branches based on the given * predicates. The predicates are applied in the given order to the values of this stream and the result * is forwarded to the corresponding (index based) result stream based on the (index of) predicate that matches. - * If none of the predicates match a value, that value is dropped. - * + *

+ * Note: If none of the predicates match a value, that value is dropped. + *

* @param predicates the predicates * @return an array of result streams (branches) corresponding to the given predicates */ diff --git a/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Sum.java b/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/LongSum.java similarity index 94% rename from storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Sum.java rename to storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/LongSum.java index df11d996b68..afc7ee7be4a 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/Sum.java +++ b/storm-core/src/jvm/org/apache/storm/streams/operations/aggregators/LongSum.java @@ -22,7 +22,7 @@ /** * Computes the long sum of the input values */ -public class Sum implements CombinerAggregator { +public class LongSum implements CombinerAggregator { @Override public Long init() { return 0L; diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateByKeyProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateByKeyProcessor.java index c10113b6965..3a7c8128eda 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateByKeyProcessor.java +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateByKeyProcessor.java @@ -22,6 +22,7 @@ import java.util.HashMap; import java.util.Map; +import java.util.function.Supplier; public class AggregateByKeyProcessor extends BaseProcessor> implements BatchProcessor { private final CombinerAggregator aggregator; @@ -47,9 +48,9 @@ public void execute(Pair input) { } state.put(key, aggregator.apply(accumulator, val)); if (emitAggregate) { - mayBeForwardAggUpdate(Pair.of(key, state.get(key))); + mayBeForwardAggUpdate(() -> Pair.of(key, state.get(key))); } else { - mayBeForwardAggUpdate(Pair.of(key, aggregator.result(state.get(key)))); + mayBeForwardAggUpdate(() -> Pair.of(key, aggregator.result(state.get(key)))); } } diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateProcessor.java index d1693453c4f..d8b78ad202f 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateProcessor.java +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/AggregateProcessor.java @@ -40,9 +40,9 @@ public void execute(T input) { } state = aggregator.apply(state, input); if (emitAggregate) { - mayBeForwardAggUpdate(state); + mayBeForwardAggUpdate(() -> state); } else { - mayBeForwardAggUpdate(aggregator.result(state)); + mayBeForwardAggUpdate(() -> aggregator.result(state)); } } diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/BaseProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/BaseProcessor.java index 3ea469c0bab..8413a2a6405 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/processors/BaseProcessor.java +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/BaseProcessor.java @@ -19,6 +19,8 @@ import java.util.HashSet; import java.util.Set; +import java.util.function.Function; +import java.util.function.Supplier; import static org.apache.storm.streams.WindowNode.PUNCTUATION; @@ -91,12 +93,12 @@ protected void finish() { * can use this to emit the partial results on each input * if they are operating in non-windowed mode. * - * @param result the result + * @param result the result function * @param the result type */ - protected final void mayBeForwardAggUpdate(R result) { + protected final void mayBeForwardAggUpdate(Supplier result) { if (!context.isWindowed()) { - context.forward(result); + context.forward(result.get()); } } diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateByKeyProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateByKeyProcessor.java index 57ad8454b55..a66c88ac185 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateByKeyProcessor.java +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateByKeyProcessor.java @@ -40,7 +40,7 @@ public void execute(Pair input) { accumulator = aggregator.init(); } state.put(key, aggregator.merge(accumulator, val)); - mayBeForwardAggUpdate(Pair.of(key, aggregator.result(state.get(key)))); + mayBeForwardAggUpdate(() -> Pair.of(key, aggregator.result(state.get(key)))); } @Override diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateProcessor.java index 61b555b4c78..9c5b1f09d5a 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateProcessor.java +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/MergeAggregateProcessor.java @@ -33,7 +33,7 @@ protected void execute(A input) { state = aggregator.init(); } state = aggregator.merge(state, input); - mayBeForwardAggUpdate(aggregator.result(state)); + mayBeForwardAggUpdate(() -> aggregator.result(state)); } @Override diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/Processor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/Processor.java index 3b4518d27e3..83daf2ac93d 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/processors/Processor.java +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/Processor.java @@ -37,7 +37,7 @@ public interface Processor extends Serializable { void init(ProcessorContext context); /** - * Executes some operation on the input and possibly emits some result. + * Executes some operations on the input and possibly emits some results. * * @param input the input to be processed * @param streamId the source stream id from where the input is received diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/ProcessorContext.java b/storm-core/src/jvm/org/apache/storm/streams/processors/ProcessorContext.java index a69a0a9fabb..45f58a17a32 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/processors/ProcessorContext.java +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/ProcessorContext.java @@ -45,10 +45,10 @@ public interface ProcessorContext extends Serializable { void forward(T input, String stream); /** - * Returns if the processing is in a windowed context and should wait for + * Returns true if the processing is in a windowed context and should wait for * punctuation before emitting results. * - * @return if this is a windowed context or not + * @return whether this is a windowed context or not */ boolean isWindowed(); diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceByKeyProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceByKeyProcessor.java index eba0de6a3fd..f20c0414a24 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceByKeyProcessor.java +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceByKeyProcessor.java @@ -36,9 +36,9 @@ public void execute(Pair input) { K key = input.getFirst(); V val = input.getSecond(); V agg = state.get(key); - agg = (agg == null) ? val : reducer.apply(agg, val); - state.put(key, agg); - mayBeForwardAggUpdate(Pair.of(key, agg)); + final V res = (agg == null) ? val : reducer.apply(agg, val); + state.put(key, res); + mayBeForwardAggUpdate(() -> Pair.of(key, res)); } @Override diff --git a/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceProcessor.java b/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceProcessor.java index 0b90fb9110e..93badf28bc9 100644 --- a/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceProcessor.java +++ b/storm-core/src/jvm/org/apache/storm/streams/processors/ReduceProcessor.java @@ -30,7 +30,7 @@ public ReduceProcessor(Reducer reducer) { @Override public void execute(T input) { agg = (agg == null) ? input : reducer.apply(agg, input); - mayBeForwardAggUpdate(agg); + mayBeForwardAggUpdate(() -> agg); } @Override diff --git a/storm-core/test/jvm/org/apache/storm/streams/ProcessorBoltTest.java b/storm-core/test/jvm/org/apache/storm/streams/ProcessorBoltTest.java index aa877f92385..93807142867 100644 --- a/storm-core/test/jvm/org/apache/storm/streams/ProcessorBoltTest.java +++ b/storm-core/test/jvm/org/apache/storm/streams/ProcessorBoltTest.java @@ -19,7 +19,7 @@ import com.google.common.collect.Multimap; import org.apache.storm.generated.GlobalStreamId; -import org.apache.storm.streams.operations.aggregators.Sum; +import org.apache.storm.streams.operations.aggregators.LongSum; import org.apache.storm.streams.processors.AggregateProcessor; import org.apache.storm.streams.processors.FilterProcessor; import org.apache.storm.streams.processors.Processor; @@ -89,7 +89,7 @@ public void testEmitAndAck() throws Exception { @Test public void testAggResultAndAck() throws Exception { - setUpProcessorBolt(new AggregateProcessor<>(new Sum()), Collections.singleton("inputstream"), true, null); + setUpProcessorBolt(new AggregateProcessor<>(new LongSum()), Collections.singleton("inputstream"), true, null); bolt.execute(mockTuple2); bolt.execute(mockTuple3); bolt.execute(punctuation); From ab430f5efa1b3df26319fef6e85b0944106d7823 Mon Sep 17 00:00:00 2001 From: Roshan Naik Date: Thu, 2 Feb 2017 16:17:15 -0800 Subject: [PATCH 07/29] STORM-2324 : Making sure toplogy deployment does not break if resources directory is missing in topology jar. UT fixes --- .../apache/storm/daemon/supervisor/Container.java | 13 ++++++++++--- .../storm/daemon/supervisor/ContainerTest.java | 2 +- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java b/storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java index e370bc4c2b9..9f416820091 100644 --- a/storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java +++ b/storm-core/src/jvm/org/apache/storm/daemon/supervisor/Container.java @@ -414,12 +414,19 @@ protected void createBlobstoreLinks() throws IOException { blobFileNames.add(ret); } } + File targetResourcesDir = new File(stormRoot, ConfigUtils.RESOURCES_SUBDIR); List resourceFileNames = new ArrayList<>(); - resourceFileNames.add(ConfigUtils.RESOURCES_SUBDIR); + if (targetResourcesDir.exists()) { + resourceFileNames.add(ConfigUtils.RESOURCES_SUBDIR); + } resourceFileNames.addAll(blobFileNames); + LOG.info("Creating symlinks for worker-id: {} storm-id: {} for files({}): {}", _workerId, _topologyId, resourceFileNames.size(), resourceFileNames); - _ops.createSymlink(new File(workerRoot, ConfigUtils.RESOURCES_SUBDIR), - new File(stormRoot, ConfigUtils.RESOURCES_SUBDIR)); + if(targetResourcesDir.exists()) { + _ops.createSymlink(new File(workerRoot, ConfigUtils.RESOURCES_SUBDIR), targetResourcesDir ); + } else { + LOG.info("Topology jar for worker-id: {} storm-id: {} does not contain re sources directory {}." , _workerId, _topologyId, targetResourcesDir.toString() ); + } for (String fileName : blobFileNames) { _ops.createSymlink(new File(workerRoot, fileName), new File(stormRoot, fileName)); diff --git a/storm-core/test/jvm/org/apache/storm/daemon/supervisor/ContainerTest.java b/storm-core/test/jvm/org/apache/storm/daemon/supervisor/ContainerTest.java index b1adcd8b50d..b6806c8e9ca 100644 --- a/storm-core/test/jvm/org/apache/storm/daemon/supervisor/ContainerTest.java +++ b/storm-core/test/jvm/org/apache/storm/daemon/supervisor/ContainerTest.java @@ -216,7 +216,7 @@ public void testSetup() throws Exception { verify(ops).createSymlink(new File(workerRoot, "artifacts"), workerArtifacts); //Create links to blobs - verify(ops).createSymlink(new File(workerRoot, "resources"), new File(distRoot, "resources")); + verify(ops, never()).createSymlink(new File(workerRoot, "resources"), new File(distRoot, "resources")); } @Test From e8c390059df5a3199763f33601b00c4a0b7d4ed0 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Fri, 3 Feb 2017 11:23:30 +0900 Subject: [PATCH 08/29] STORM-2324: CHANGELOG --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 275e32f4245..e5a6fbafadb 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -196,6 +196,7 @@ * STORM-1769: Added a test to check local nimbus with notifier plugin ## 1.1.0 + * STORM-2324: Fix deployment failure if resources directory is missing in topology jar * STORM-1443: [Storm SQL] Support customizing parallelism in StormSQL * STORM-2148: [Storm SQL] Trident mode: back to code generate and compile Trident topology * STORM-2321: Handle blobstore zk key deletion in KeySequenceNumber. From 38ee403bf4a537ef276221033901690f099479c0 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Fri, 3 Feb 2017 11:52:02 +0900 Subject: [PATCH 09/29] STORM-2014: CHANGELOG --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index e5a6fbafadb..37391149fd7 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -196,6 +196,7 @@ * STORM-1769: Added a test to check local nimbus with notifier plugin ## 1.1.0 + * STORM-2014: New Kafka spout duplicates checking if failed messages have reached max retries * STORM-2324: Fix deployment failure if resources directory is missing in topology jar * STORM-1443: [Storm SQL] Support customizing parallelism in StormSQL * STORM-2148: [Storm SQL] Trident mode: back to code generate and compile Trident topology From bd6e0c5fbcebd1f3ac99e4095c3c492c39873455 Mon Sep 17 00:00:00 2001 From: Ernestas Vaiciukevicius Date: Thu, 12 Jan 2017 16:54:59 +0200 Subject: [PATCH 10/29] STORM-2296 Kafka spout no dup on leader changes --- .../apache/storm/kafka/PartitionManager.java | 125 +++++++++++------- .../org/apache/storm/kafka/ZkCoordinator.java | 16 ++- .../apache/storm/kafka/ZkCoordinatorTest.java | 42 +++++- 3 files changed, 131 insertions(+), 52 deletions(-) diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java index 8d608d9bf0a..e4ce657f21a 100644 --- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java +++ b/external/storm-kafka/src/jvm/org/apache/storm/kafka/PartitionManager.java @@ -66,7 +66,29 @@ public class PartitionManager { ZkState _state; Map _stormConf; long numberFailed, numberAcked; - public PartitionManager(DynamicPartitionConnections connections, String topologyInstanceId, ZkState state, Map stormConf, SpoutConfig spoutConfig, Partition id) { + + public PartitionManager( + DynamicPartitionConnections connections, + String topologyInstanceId, + ZkState state, + Map stormConf, + SpoutConfig spoutConfig, + Partition id) + { + this(connections, topologyInstanceId, state, stormConf, spoutConfig, id, null); + } + + /** + * @param previousManager previous partition manager if manager for partition is being recreated + */ + public PartitionManager( + DynamicPartitionConnections connections, + String topologyInstanceId, + ZkState state, + Map stormConf, + SpoutConfig spoutConfig, + Partition id, + PartitionManager previousManager) { _partition = id; _connections = connections; _spoutConfig = spoutConfig; @@ -76,53 +98,64 @@ public PartitionManager(DynamicPartitionConnections connections, String topology _stormConf = stormConf; numberAcked = numberFailed = 0; - try { - _failedMsgRetryManager = (FailedMsgRetryManager) Class.forName(spoutConfig.failedMsgRetryManagerClass).newInstance(); - _failedMsgRetryManager.prepare(spoutConfig, _stormConf); - } catch (ClassNotFoundException | InstantiationException | IllegalAccessException e) { - throw new IllegalArgumentException(String.format("Failed to create an instance of <%s> from: <%s>", - FailedMsgRetryManager.class, - spoutConfig.failedMsgRetryManagerClass), e); - } + if (previousManager != null) { + _failedMsgRetryManager = previousManager._failedMsgRetryManager; + _committedTo = previousManager._committedTo; + _emittedToOffset = previousManager._emittedToOffset; + _waitingToEmit = previousManager._waitingToEmit; + _pending = previousManager._pending; + LOG.info("Recreating PartitionManager based on previous manager, _waitingToEmit size: {}, _pending size: {}", + _waitingToEmit.size(), + _pending.size()); + } else { + try { + _failedMsgRetryManager = (FailedMsgRetryManager) Class.forName(spoutConfig.failedMsgRetryManagerClass).newInstance(); + _failedMsgRetryManager.prepare(spoutConfig, _stormConf); + } catch (ClassNotFoundException | InstantiationException | IllegalAccessException e) { + throw new IllegalArgumentException(String.format("Failed to create an instance of <%s> from: <%s>", + FailedMsgRetryManager.class, + spoutConfig.failedMsgRetryManagerClass), e); + } - String jsonTopologyId = null; - Long jsonOffset = null; - String path = committedPath(); - try { - Map json = _state.readJSON(path); - LOG.info("Read partition information from: " + path + " --> " + json ); - if (json != null) { - jsonTopologyId = (String) ((Map) json.get("topology")).get("id"); - jsonOffset = (Long) json.get("offset"); + String jsonTopologyId = null; + Long jsonOffset = null; + String path = committedPath(); + try { + Map json = _state.readJSON(path); + LOG.info("Read partition information from: " + path + " --> " + json); + if (json != null) { + jsonTopologyId = (String) ((Map) json.get("topology")).get("id"); + jsonOffset = (Long) json.get("offset"); + } + } catch (Throwable e) { + LOG.warn("Error reading and/or parsing at ZkNode: " + path, e); } - } catch (Throwable e) { - LOG.warn("Error reading and/or parsing at ZkNode: " + path, e); - } - String topic = _partition.topic; - Long currentOffset = KafkaUtils.getOffset(_consumer, topic, id.partition, spoutConfig); + String topic = _partition.topic; + Long currentOffset = KafkaUtils.getOffset(_consumer, topic, id.partition, spoutConfig); - if (jsonTopologyId == null || jsonOffset == null) { // failed to parse JSON? - _committedTo = currentOffset; - LOG.info("No partition information found, using configuration to determine offset"); - } else if (!topologyInstanceId.equals(jsonTopologyId) && spoutConfig.ignoreZkOffsets) { - _committedTo = KafkaUtils.getOffset(_consumer, topic, id.partition, spoutConfig.startOffsetTime); - LOG.info("Topology change detected and ignore zookeeper offsets set to true, using configuration to determine offset"); - } else { - _committedTo = jsonOffset; - LOG.info("Read last commit offset from zookeeper: " + _committedTo + "; old topology_id: " + jsonTopologyId + " - new topology_id: " + topologyInstanceId ); - } + if (jsonTopologyId == null || jsonOffset == null) { // failed to parse JSON? + _committedTo = currentOffset; + LOG.info("No partition information found, using configuration to determine offset"); + } else if (!topologyInstanceId.equals(jsonTopologyId) && spoutConfig.ignoreZkOffsets) { + _committedTo = KafkaUtils.getOffset(_consumer, topic, id.partition, spoutConfig.startOffsetTime); + LOG.info("Topology change detected and ignore zookeeper offsets set to true, using configuration to determine offset"); + } else { + _committedTo = jsonOffset; + LOG.info("Read last commit offset from zookeeper: " + _committedTo + "; old topology_id: " + jsonTopologyId + " - new topology_id: " + topologyInstanceId); + } - if (currentOffset - _committedTo > spoutConfig.maxOffsetBehind || _committedTo <= 0) { - LOG.info("Last commit offset from zookeeper: " + _committedTo); - Long lastCommittedOffset = _committedTo; - _committedTo = currentOffset; - LOG.info("Commit offset " + lastCommittedOffset + " is more than " + - spoutConfig.maxOffsetBehind + " behind latest offset " + currentOffset + ", resetting to startOffsetTime=" + spoutConfig.startOffsetTime); - } + if (currentOffset - _committedTo > spoutConfig.maxOffsetBehind || _committedTo <= 0) { + LOG.info("Last commit offset from zookeeper: " + _committedTo); + Long lastCommittedOffset = _committedTo; + _committedTo = currentOffset; + LOG.info("Commit offset " + lastCommittedOffset + " is more than " + + spoutConfig.maxOffsetBehind + " behind latest offset " + currentOffset + ", resetting to startOffsetTime=" + spoutConfig.startOffsetTime); + } - LOG.info("Starting Kafka " + _consumer.host() + " " + id + " from offset " + _committedTo); - _emittedToOffset = _committedTo; + LOG.info("Starting Kafka " + _consumer.host() + " " + id + " from offset " + _committedTo); + _emittedToOffset = _committedTo; + } _fetchAPILatencyMax = new CombinedMetric(new MaxMetric()); _fetchAPILatencyMean = new ReducedMetric(new MeanReducer()); @@ -160,7 +193,7 @@ public EmitState next(SpoutOutputCollector collector) { } else { tups = KafkaUtils.generateTuples(_spoutConfig, toEmit.message(), _partition.topic); } - + if ((tups != null) && tups.iterator().hasNext()) { if (!Strings.isNullOrEmpty(_spoutConfig.outputStreamId)) { for (List tup : tups) { @@ -201,7 +234,7 @@ private void fill() { } catch (TopicOffsetOutOfRangeException e) { offset = KafkaUtils.getOffset(_consumer, _partition.topic, _partition.partition, kafka.api.OffsetRequest.EarliestTime()); // fetch failed, so don't update the fetch metrics - + //fix bug [STORM-643] : remove outdated failed offsets if (!processingNewTuples) { // For the case of EarliestTime it would be better to discard @@ -214,7 +247,7 @@ private void fill() { if (null != omitted) { _lostMessageCount.incrBy(omitted.size()); } - + LOG.warn("Removing the failed offsets for {} that are out of range: {}", _partition, omitted); } @@ -223,7 +256,7 @@ private void fill() { _emittedToOffset = offset; LOG.warn("{} Using new offset: {}", _partition, _emittedToOffset); } - + return; } long millis = System.currentTimeMillis() - start; diff --git a/external/storm-kafka/src/jvm/org/apache/storm/kafka/ZkCoordinator.java b/external/storm-kafka/src/jvm/org/apache/storm/kafka/ZkCoordinator.java index 98bf8a0fabc..14be5845007 100644 --- a/external/storm-kafka/src/jvm/org/apache/storm/kafka/ZkCoordinator.java +++ b/external/storm-kafka/src/jvm/org/apache/storm/kafka/ZkCoordinator.java @@ -88,14 +88,24 @@ public void refresh() { LOG.info(taskId(_taskIndex, _totalTasks) + "Deleted partition managers: " + deletedPartitions.toString()); + Map deletedManagers = new HashMap<>(); for (Partition id : deletedPartitions) { - PartitionManager man = _managers.remove(id); - man.close(); + deletedManagers.put(id.partition, _managers.remove(id)); + } + for (PartitionManager manager : deletedManagers.values()) { + if (manager != null) manager.close(); } LOG.info(taskId(_taskIndex, _totalTasks) + "New partition managers: " + newPartitions.toString()); for (Partition id : newPartitions) { - PartitionManager man = new PartitionManager(_connections, _topologyInstanceId, _state, _stormConf, _spoutConfig, id); + PartitionManager man = new PartitionManager( + _connections, + _topologyInstanceId, + _state, + _stormConf, + _spoutConfig, + id, + deletedManagers.get(id.partition)); _managers.put(id, man); } diff --git a/external/storm-kafka/src/test/org/apache/storm/kafka/ZkCoordinatorTest.java b/external/storm-kafka/src/test/org/apache/storm/kafka/ZkCoordinatorTest.java index 364da3314f3..b23d5bcea2b 100644 --- a/external/storm-kafka/src/test/org/apache/storm/kafka/ZkCoordinatorTest.java +++ b/external/storm-kafka/src/test/org/apache/storm/kafka/ZkCoordinatorTest.java @@ -28,8 +28,7 @@ import java.util.*; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.*; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyInt; import static org.mockito.Mockito.when; @@ -106,7 +105,7 @@ public void testPartitionsChange() throws Exception { waitForRefresh(); when(reader.getBrokerInfo()).thenReturn(TestUtils.buildPartitionInfoList(TestUtils.buildPartitionInfo(totalTasks, 9093))); List> partitionManagersAfterRefresh = getPartitionManagers(coordinatorList); - assertEquals(partitionManagersAfterRefresh.size(), partitionManagersAfterRefresh.size()); + assertEquals(partitionManagersBeforeRefresh.size(), partitionManagersAfterRefresh.size()); Iterator> iterator = partitionManagersAfterRefresh.iterator(); for (List partitionManagersBefore : partitionManagersBeforeRefresh) { List partitionManagersAfter = iterator.next(); @@ -114,6 +113,43 @@ public void testPartitionsChange() throws Exception { } } + @Test + public void testPartitionManagerRecreate() throws Exception { + final int totalTasks = 2; + int partitionsPerTask = 2; + List coordinatorList = buildCoordinators(totalTasks / partitionsPerTask); + when(reader.getBrokerInfo()).thenReturn(TestUtils.buildPartitionInfoList(TestUtils.buildPartitionInfo(totalTasks, 9092))); + List> partitionManagersBeforeRefresh = getPartitionManagers(coordinatorList); + waitForRefresh(); + when(reader.getBrokerInfo()).thenReturn(TestUtils.buildPartitionInfoList(TestUtils.buildPartitionInfo(totalTasks, 9093))); + List> partitionManagersAfterRefresh = getPartitionManagers(coordinatorList); + assertEquals(partitionManagersBeforeRefresh.size(), partitionManagersAfterRefresh.size()); + + HashMap managersAfterRefresh = new HashMap(); + for (List partitionManagersAfter : partitionManagersAfterRefresh) { + for (PartitionManager manager : partitionManagersAfter) { + assertFalse("Multiple PartitionManagers for same partition", managersAfterRefresh.containsKey(manager.getPartition().partition)); + managersAfterRefresh.put(manager.getPartition().partition, manager); + } + } + + for (List partitionManagersBefore : partitionManagersBeforeRefresh) { + for (PartitionManager manager : partitionManagersBefore) { + assertStateIsTheSame(manager, managersAfterRefresh.get(manager.getPartition().partition)); + } + } + } + + private void assertStateIsTheSame(PartitionManager managerBefore, PartitionManager managerAfter) { + // check if state was actually moved from old PartitionManager + assertNotNull(managerBefore); + assertNotNull(managerAfter); + assertNotSame(managerBefore, managerAfter); + assertSame(managerBefore._waitingToEmit, managerAfter._waitingToEmit); + assertSame(managerBefore._emittedToOffset, managerAfter._emittedToOffset); + assertSame(managerBefore._committedTo, managerAfter._committedTo); + } + private void assertPartitionsAreDifferent(List partitionManagersBefore, List partitionManagersAfter, int partitionsPerTask) { assertEquals(partitionsPerTask, partitionManagersBefore.size()); assertEquals(partitionManagersBefore.size(), partitionManagersAfter.size()); From 5d8f23026e1e19c37603252b12721b4d9f158c9c Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Sat, 4 Feb 2017 08:24:31 +0900 Subject: [PATCH 11/29] STORM-2296: CHANGELOG --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 37391149fd7..c4271d07682 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -196,6 +196,7 @@ * STORM-1769: Added a test to check local nimbus with notifier plugin ## 1.1.0 + * STORM-2296: Kafka spout no dup on leader changes * STORM-2014: New Kafka spout duplicates checking if failed messages have reached max retries * STORM-2324: Fix deployment failure if resources directory is missing in topology jar * STORM-1443: [Storm SQL] Support customizing parallelism in StormSQL From a7cdfefd7dc60df6c257c78f5dc908e815fd0812 Mon Sep 17 00:00:00 2001 From: vesense Date: Sat, 4 Feb 2017 10:39:24 +0800 Subject: [PATCH 12/29] STORM-2337: CHANGELOG --- CHANGELOG.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index c4271d07682..3c41ba3290e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,4 @@ -## 2.0.0 +## 2.0.0 * STORM-2327: Introduce ConfigurableTopology * STORM-2323: Precondition for Leader Nimbus should check all topology blobs and also corresponding dependencies. * STORM-2305: STORM-2279 calculates task index different from grouper code @@ -304,6 +304,7 @@ * STORM-1868: Modify TridentKafkaWordCount to run in distributed mode ## 1.0.3 + * STORM-2337: Broken documentation generation for storm-metrics-profiling-internal-actions.md and windows-users-guide.md * STORM-2325: Logviewer doesn't consider 'storm.local.hostname' * STORM-1742: More accurate 'complete latency' * STORM-2176: Workers do not shutdown cleanly and worker hooks don't run when a topology is killed From 01d7b6d081db98c284ed16036d0159faee2d0946 Mon Sep 17 00:00:00 2001 From: Xin Wang Date: Mon, 6 Feb 2017 10:17:25 +0800 Subject: [PATCH 13/29] STORM-2346: add missing licenses --- dev-tools/rc/download-rc-directory.sh | 11 +++++++++++ dev-tools/rc/verify-release-file.sh | 11 +++++++++++ 2 files changed, 22 insertions(+) diff --git a/dev-tools/rc/download-rc-directory.sh b/dev-tools/rc/download-rc-directory.sh index 196a7844fee..c5d674a7fb4 100755 --- a/dev-tools/rc/download-rc-directory.sh +++ b/dev-tools/rc/download-rc-directory.sh @@ -1,4 +1,15 @@ #!/bin/bash +# Licensed 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. TARGET_URL=$1 diff --git a/dev-tools/rc/verify-release-file.sh b/dev-tools/rc/verify-release-file.sh index 56b5c3173d0..dcb2202bd64 100755 --- a/dev-tools/rc/verify-release-file.sh +++ b/dev-tools/rc/verify-release-file.sh @@ -1,4 +1,15 @@ #!/bin/bash +# Licensed 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. TARGET_FILE=$1 From b5f02d4e4cac577454c379ca4e150c7ae47307be Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Tue, 7 Feb 2017 10:27:56 +0900 Subject: [PATCH 14/29] STORM-1961: CHANGELOG --- CHANGELOG.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 3c41ba3290e..8dc0c8af59b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ -## 2.0.0 +## 2.0.0 + * STORM-1961: Stream api for storm core use cases * STORM-2327: Introduce ConfigurableTopology * STORM-2323: Precondition for Leader Nimbus should check all topology blobs and also corresponding dependencies. * STORM-2305: STORM-2279 calculates task index different from grouper code From 5be2ce8afb33bff926547da6034f16cdc909f4a0 Mon Sep 17 00:00:00 2001 From: Tibor Kiss Date: Fri, 3 Feb 2017 09:10:10 +0100 Subject: [PATCH 15/29] STORM-2338: Subprocess exception handling is broken in storm.py on Windows environment --- bin/storm.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/storm.py b/bin/storm.py index 29802fee236..7ab1f9c85f0 100755 --- a/bin/storm.py +++ b/bin/storm.py @@ -260,7 +260,7 @@ def exec_storm_class(klass, jvmtype="-server", jvmopts=[], extrajars=[], args=[] try: ret = sub.check_output(all_args, stderr=sub.STDOUT) print(ret) - except sub.CalledProcessor as e: + except sub.CalledProcessError as e: sys.exit(e.returncode) else: os.execvp(JAVA_CMD, all_args) From b705d861b925e0a8e486449f01c594150021aae7 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Tue, 7 Feb 2017 11:14:41 +0900 Subject: [PATCH 16/29] STORM-2338: CHANGELOG, reflect backport STORM-2197, STORM-2324, STORM-2321, STORM-2326, STORM-2335 --- CHANGELOG.md | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 8dc0c8af59b..e4e1f359be9 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -29,7 +29,6 @@ * storm-2205: Racecondition in getting nimbus summaries while ZK connections are reconnected * STORM-1278: Port org.apache.storm.daemon.worker to java * STORM-2192: Add a new IAutoCredentials plugin to support SSL files - * STORM-2197: NimbusClient connectins leak due to leakage in ThriftClient * STORM-2185: Storm Supervisor doesn't delete directories properly sometimes * STORM-2188: Interrupt all executor threads before joining in executor shutdown * STORM-203: Adding paths to default java library path @@ -199,12 +198,8 @@ ## 1.1.0 * STORM-2296: Kafka spout no dup on leader changes * STORM-2014: New Kafka spout duplicates checking if failed messages have reached max retries - * STORM-2324: Fix deployment failure if resources directory is missing in topology jar * STORM-1443: [Storm SQL] Support customizing parallelism in StormSQL * STORM-2148: [Storm SQL] Trident mode: back to code generate and compile Trident topology - * STORM-2321: Handle blobstore zk key deletion in KeySequenceNumber. - * STORM-2336: Close Localizer and AsyncLocalizer when supervisor is shutting down - * STORM-2335: Fix broken Topology visualization with empty ':transferred' in executor stats * STORM-2331: Emitting from JavaScript should work when not anchoring. * STORM-2320: DRPC client printer class reusable for local and remote DRPC. * STORM-2225: change spout config to be simpler. @@ -305,6 +300,12 @@ * STORM-1868: Modify TridentKafkaWordCount to run in distributed mode ## 1.0.3 + * STORM-2197: NimbusClient connectins leak due to leakage in ThriftClient + * STORM-2321: Handle blobstore zk key deletion in KeySequenceNumber. + * STORM-2324: Fix deployment failure if resources directory is missing in topology jar + * STORM-2335: Fix broken Topology visualization with empty ':transferred' in executor stats + * STORM-2336: Close Localizer and AsyncLocalizer when supervisor is shutting down + * STORM-2338: Subprocess exception handling is broken in storm.py on Windows environment * STORM-2337: Broken documentation generation for storm-metrics-profiling-internal-actions.md and windows-users-guide.md * STORM-2325: Logviewer doesn't consider 'storm.local.hostname' * STORM-1742: More accurate 'complete latency' From d3250b22f17bf98ffe97b00c9bdd2cdf0dc9eb8c Mon Sep 17 00:00:00 2001 From: vesense Date: Tue, 7 Feb 2017 14:44:16 +0800 Subject: [PATCH 17/29] STORM-2346: CHANGELOG --- CHANGELOG.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index e4e1f359be9..e705c9349cd 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ -## 2.0.0 +## 2.0.0 + * STORM-2346: Files with unapproved licenses: download-rc-directory.sh verify-release-file.sh * STORM-1961: Stream api for storm core use cases * STORM-2327: Introduce ConfigurableTopology * STORM-2323: Precondition for Leader Nimbus should check all topology blobs and also corresponding dependencies. From 07cf86e95a66e47cac41f67f5b269ab24758d8cb Mon Sep 17 00:00:00 2001 From: Hugo Louro Date: Mon, 30 Jan 2017 00:02:42 -0800 Subject: [PATCH 18/29] STORM-2281: Running Multiple Kafka Spouts (Trident) Throws Illegal State Exception - Assign topic partitions to tasks running the instance of Kafka consumer that has assigned the same list of topic partitions - Improve logging - Minor code refactoring --- ...ridentKafkaClientWordCountNamedTopics.java | 44 +++- .../storm/kafka/trident/LocalSubmitter.java | 7 +- .../trident/TridentKafkaConsumerTopology.java | 20 +- .../KafkaTridentSpoutBatchMetadata.java | 8 +- .../trident/KafkaTridentSpoutEmitter.java | 206 ++++++++++++------ .../trident/KafkaTridentSpoutManager.java | 85 ++++---- .../trident/KafkaTridentSpoutOpaque.java | 17 +- .../KafkaTridentSpoutOpaqueCoordinator.java | 4 +- .../KafkaTridentSpoutTopicPartition.java | 2 +- ...fkaTridentSpoutTopicPartitionRegistry.java | 4 +- 10 files changed, 237 insertions(+), 160 deletions(-) diff --git a/examples/storm-kafka-client-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaClientWordCountNamedTopics.java b/examples/storm-kafka-client-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaClientWordCountNamedTopics.java index 2d08f6c5e88..edd1f099f0e 100644 --- a/examples/storm-kafka-client-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaClientWordCountNamedTopics.java +++ b/examples/storm-kafka-client-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaClientWordCountNamedTopics.java @@ -18,16 +18,13 @@ package org.apache.storm.kafka.trident; -import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST; - -import java.util.Arrays; -import java.util.concurrent.TimeUnit; - +import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.storm.Config; import org.apache.storm.StormSubmitter; import org.apache.storm.generated.AlreadyAliveException; import org.apache.storm.generated.AuthorizationException; import org.apache.storm.generated.InvalidTopologyException; +import org.apache.storm.kafka.spout.Func; import org.apache.storm.kafka.spout.KafkaSpoutConfig; import org.apache.storm.kafka.spout.KafkaSpoutRetryExponentialBackoff; import org.apache.storm.kafka.spout.KafkaSpoutRetryExponentialBackoff.TimeInterval; @@ -36,6 +33,13 @@ import org.apache.storm.tuple.Fields; import org.apache.storm.tuple.Values; +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST; + public class TridentKafkaClientWordCountNamedTopics { private static final String TOPIC_1 = "test-trident"; private static final String TOPIC_2 = "test-trident-1"; @@ -45,11 +49,23 @@ private KafkaTridentSpoutOpaque newKafkaTridentSpoutOpaque() { return new KafkaTridentSpoutOpaque<>(newKafkaSpoutConfig()); } + private static Func, List> JUST_VALUE_FUNC = new JustValueFunc(); + + /** + * Needs to be serializable + */ + private static class JustValueFunc implements Func, List>, Serializable { + @Override + public List apply(ConsumerRecord record) { + return new Values(record.value()); + } + } + protected KafkaSpoutConfig newKafkaSpoutConfig() { return KafkaSpoutConfig.builder(KAFKA_LOCAL_BROKER, TOPIC_1, TOPIC_2) - .setGroupId("kafkaSpoutTestGroup") + .setGroupId("kafkaSpoutTestGroup_" + System.nanoTime()) .setMaxPartitionFectchBytes(200) - .setRecordTranslator((r) -> new Values(r.value()), new Fields("str")) + .setRecordTranslator(JUST_VALUE_FUNC, new Fields("str")) .setRetry(newRetryService()) .setOffsetCommitPeriodMs(10_000) .setFirstPollOffsetStrategy(EARLIEST) @@ -77,7 +93,7 @@ protected void run(String[] args) throws AlreadyAliveException, InvalidTopologyE System.out.printf("Running with broker_url: [%s], topics: [%s, %s]\n", brokerUrl, topic1, topic2); - Config tpConf = LocalSubmitter.defaultConfig(); + Config tpConf = LocalSubmitter.defaultConfig(true); if (args.length == 4) { //Submit Remote // Producers @@ -102,11 +118,15 @@ protected void run(String[] args) throws AlreadyAliveException, InvalidTopologyE localSubmitter.submit(topic1Tp, tpConf, KafkaProducerTopology.newTopology(brokerUrl, topic1)); localSubmitter.submit(topic2Tp, tpConf, KafkaProducerTopology.newTopology(brokerUrl, topic2)); // Consumer - localSubmitter.submit(consTpName, tpConf, TridentKafkaConsumerTopology.newTopology( - localSubmitter.getDrpc(), newKafkaTridentSpoutOpaque())); + try { + localSubmitter.submit(consTpName, tpConf, TridentKafkaConsumerTopology.newTopology( + localSubmitter.getDrpc(), newKafkaTridentSpoutOpaque())); + // print + localSubmitter.printResults(15, 1, TimeUnit.SECONDS); + } catch (Exception e) { + e.printStackTrace(); + } - // print - new DrpcResultsPrinter(localSubmitter.getDrpc()).printResults(60, 1, TimeUnit.SECONDS); } finally { // kill localSubmitter.kill(topic1Tp); diff --git a/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/LocalSubmitter.java b/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/LocalSubmitter.java index 54ec99c9be6..96666957edd 100644 --- a/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/LocalSubmitter.java +++ b/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/LocalSubmitter.java @@ -47,10 +47,13 @@ public static LocalSubmitter newInstance() { } public static Config defaultConfig() { + return defaultConfig(false); + } + + public static Config defaultConfig(boolean debug) { final Config conf = new Config(); conf.setMaxSpoutPending(20); - conf.setMaxTaskParallelism(1); - conf.setNumWorkers(1); + conf.setDebug(debug); return conf; } diff --git a/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaConsumerTopology.java b/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaConsumerTopology.java index 4669f52d13b..a39eba1a5d3 100644 --- a/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaConsumerTopology.java +++ b/examples/storm-kafka-examples/src/main/java/org/apache/storm/kafka/trident/TridentKafkaConsumerTopology.java @@ -18,9 +18,7 @@ package org.apache.storm.kafka.trident; -import org.apache.storm.Config; import org.apache.storm.LocalDRPC; -import org.apache.storm.StormSubmitter; import org.apache.storm.generated.StormTopology; import org.apache.storm.starter.trident.DebugMemoryMapState; import org.apache.storm.trident.Stream; @@ -42,14 +40,6 @@ public class TridentKafkaConsumerTopology { protected static final Logger LOG = LoggerFactory.getLogger(TridentKafkaConsumerTopology.class); - public static void submitRemote(String name, ITridentDataSource tridentSpout) { - try { - StormSubmitter.submitTopology(name, newTpConfig(), newTopology(null, tridentSpout)); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - /** * See {@link TridentKafkaConsumerTopology#newTopology(LocalDRPC, ITridentDataSource)} */ @@ -85,19 +75,11 @@ public boolean isKeep(TridentTuple tuple) { } private static TridentState addTridentState(TridentTopology tridentTopology, ITridentDataSource tridentSpout) { - final Stream spoutStream = tridentTopology.newStream("spout1", tridentSpout).parallelismHint(1); + final Stream spoutStream = tridentTopology.newStream("spout1", tridentSpout).parallelismHint(2); return spoutStream.each(spoutStream.getOutputFields(), new Debug(true)) .each(new Fields("str"), new Split(), new Fields("word")) .groupBy(new Fields("word")) .persistentAggregate(new DebugMemoryMapState.Factory(), new Count(), new Fields("count")); } - - private static Config newTpConfig() { - Config conf = new Config(); - conf.setMaxSpoutPending(20); - conf.setMaxTaskParallelism(1); - return conf; - } - } diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutBatchMetadata.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutBatchMetadata.java index 2faf52a2dd9..18a22467ec4 100644 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutBatchMetadata.java +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutBatchMetadata.java @@ -34,8 +34,8 @@ public class KafkaTridentSpoutBatchMetadata implements Serializable { private static final Logger LOG = LoggerFactory.getLogger(KafkaTridentSpoutBatchMetadata.class); private TopicPartition topicPartition; // topic partition of this batch - private long firstOffset; // first offset of this batch - private long lastOffset; // last offset of this batch + private long firstOffset; // first offset of this batch + private long lastOffset; // last offset of this batch public KafkaTridentSpoutBatchMetadata(TopicPartition topicPartition, long firstOffset, long lastOffset) { this.topicPartition = topicPartition; @@ -74,8 +74,8 @@ public TopicPartition getTopicPartition() { @Override public String toString() { - return "KafkaTridentSpoutBatchMetadata{" + - "topicPartition=" + topicPartition + + return super.toString() + + "{topicPartition=" + topicPartition + ", firstOffset=" + firstOffset + ", lastOffset=" + lastOffset + '}'; diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutEmitter.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutEmitter.java index 19b4f01fdac..79dfc606815 100644 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutEmitter.java +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutEmitter.java @@ -18,20 +18,6 @@ package org.apache.storm.kafka.spout.trident; -import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST; -import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.LATEST; -import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.UNCOMMITTED_EARLIEST; -import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.UNCOMMITTED_LATEST; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.concurrent.TimeUnit; - import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -47,9 +33,28 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class KafkaTridentSpoutEmitter implements IOpaquePartitionedTridentSpout.Emitter, KafkaTridentSpoutTopicPartition, KafkaTridentSpoutBatchMetadata>, Serializable { - private static final long serialVersionUID = -7343927794834130435L; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST; +import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.LATEST; +import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.UNCOMMITTED_EARLIEST; +import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.UNCOMMITTED_LATEST; +public class KafkaTridentSpoutEmitter implements IOpaquePartitionedTridentSpout.Emitter< + List, + KafkaTridentSpoutTopicPartition, + KafkaTridentSpoutBatchMetadata>, + Serializable { + + private static final long serialVersionUID = -7343927794834130435L; private static final Logger LOG = LoggerFactory.getLogger(KafkaTridentSpoutEmitter.class); // Kafka @@ -57,60 +62,81 @@ public class KafkaTridentSpoutEmitter implements IOpaquePartitionedTridentS // Bookkeeping private final KafkaTridentSpoutManager kafkaManager; + // Declare some KafkaTridentSpoutManager references for convenience private final long pollTimeoutMs; private final KafkaSpoutConfig.FirstPollOffsetStrategy firstPollOffsetStrategy; private final RecordTranslator translator; private final Timer refreshSubscriptionTimer; - public KafkaTridentSpoutEmitter(KafkaTridentSpoutManager kafkaManager, TopologyContext context) { - this.kafkaManager = kafkaManager; - this.kafkaManager.subscribeKafkaConsumer(context); - refreshSubscriptionTimer = new Timer(500, kafkaManager.getKafkaSpoutConfig().getPartitionRefreshPeriodMs(), TimeUnit.MILLISECONDS); + private TopologyContext topologyContext; - //must subscribeKafkaConsumer before this line - kafkaConsumer = kafkaManager.getKafkaConsumer(); - translator = kafkaManager.getKafkaSpoutConfig().getTranslator(); + public KafkaTridentSpoutEmitter(KafkaTridentSpoutManager kafkaManager, TopologyContext topologyContext, Timer refreshSubscriptionTimer) { + this.kafkaConsumer = kafkaManager.createAndSubscribeKafkaConsumer(topologyContext); + this.kafkaManager = kafkaManager; + this.topologyContext = topologyContext; + this.refreshSubscriptionTimer = refreshSubscriptionTimer; + this.translator = kafkaManager.getKafkaSpoutConfig().getTranslator(); final KafkaSpoutConfig kafkaSpoutConfig = kafkaManager.getKafkaSpoutConfig(); - pollTimeoutMs = kafkaSpoutConfig.getPollTimeoutMs(); - firstPollOffsetStrategy = kafkaSpoutConfig.getFirstPollOffsetStrategy(); + this.pollTimeoutMs = kafkaSpoutConfig.getPollTimeoutMs(); + this.firstPollOffsetStrategy = kafkaSpoutConfig.getFirstPollOffsetStrategy(); LOG.debug("Created {}", this); } + /** + * Creates instance of this class with default 500 millisecond refresh subscription timer + */ + public KafkaTridentSpoutEmitter(KafkaTridentSpoutManager kafkaManager, TopologyContext topologyContext) { + this(kafkaManager, topologyContext, new Timer(500, + kafkaManager.getKafkaSpoutConfig().getPartitionRefreshPeriodMs(), TimeUnit.MILLISECONDS)); + } + @Override public KafkaTridentSpoutBatchMetadata emitPartitionBatch(TransactionAttempt tx, TridentCollector collector, - KafkaTridentSpoutTopicPartition partitionTs, KafkaTridentSpoutBatchMetadata lastBatch) { - LOG.debug("Emitting batch: [transaction = {}], [partition = {}], [collector = {}], [lastBatchMetadata = {}]", - tx, partitionTs, collector, lastBatch); + KafkaTridentSpoutTopicPartition currBatchPartition, KafkaTridentSpoutBatchMetadata lastBatch) { - final TopicPartition topicPartition = partitionTs.getTopicPartition(); + LOG.debug("Processing batch: [transaction = {}], [currBatchPartition = {}], [lastBatchMetadata = {}], [collector = {}]", + tx, currBatchPartition, lastBatch, collector); + + final TopicPartition currBatchTp = currBatchPartition.getTopicPartition(); + final Set assignments = kafkaConsumer.assignment(); KafkaTridentSpoutBatchMetadata currentBatch = lastBatch; Collection pausedTopicPartitions = Collections.emptySet(); - try { - // pause other topic partitions to only poll from current topic partition - pausedTopicPartitions = pauseTopicPartitions(topicPartition); + if (assignments == null || !assignments.contains(currBatchPartition.getTopicPartition())) { + LOG.warn("SKIPPING processing batch: [transaction = {}], [currBatchPartition = {}], [lastBatchMetadata = {}], " + + "[collector = {}] because it is not assigned {} to consumer instance [{}] of consumer group [{}]", + tx, currBatchPartition, lastBatch, collector, assignments, kafkaConsumer, + kafkaManager.getKafkaSpoutConfig().getConsumerGroupId()); + } else { + try { + // pause other topic-partitions to only poll from current topic-partition + pausedTopicPartitions = pauseTopicPartitions(currBatchTp); - seek(topicPartition, lastBatch); + seek(currBatchTp, lastBatch); - // poll - if (refreshSubscriptionTimer.isExpiredResetOnTrue()) { - kafkaManager.getKafkaSpoutConfig().getSubscription().refreshAssignment(); - } - final ConsumerRecords records = kafkaConsumer.poll(pollTimeoutMs); - LOG.debug("Polled [{}] records from Kafka.", records.count()); + // poll + if (refreshSubscriptionTimer.isExpiredResetOnTrue()) { + kafkaManager.getKafkaSpoutConfig().getSubscription().refreshAssignment(); + } + + final ConsumerRecords records = kafkaConsumer.poll(pollTimeoutMs); + LOG.debug("Polled [{}] records from Kafka.", records.count()); - if (!records.isEmpty()) { - emitTuples(collector, records); - // build new metadata - currentBatch = new KafkaTridentSpoutBatchMetadata<>(topicPartition, records, lastBatch); + if (!records.isEmpty()) { + emitTuples(collector, records); + // build new metadata + currentBatch = new KafkaTridentSpoutBatchMetadata<>(currBatchTp, records, lastBatch); + } + } finally { + kafkaConsumer.resume(pausedTopicPartitions); + LOG.trace("Resumed topic-partitions {}", pausedTopicPartitions); } - } finally { - kafkaConsumer.resume(pausedTopicPartitions); - LOG.trace("Resumed topic partitions [{}]", pausedTopicPartitions); + LOG.debug("Emitted batch: [transaction = {}], [currBatchPartition = {}], [lastBatchMetadata = {}], " + + "[currBatchMetadata = {}], [collector = {}]", tx, currBatchPartition, lastBatch, currentBatch, collector); } - LOG.debug("Current batch metadata {}", currentBatch); + return currentBatch; } @@ -118,7 +144,7 @@ private void emitTuples(TridentCollector collector, ConsumerRecords record for (ConsumerRecord record : records) { final List tuple = translator.apply(record); collector.emit(tuple); - LOG.debug("Emitted tuple [{}] for record: [{}]", tuple, record); + LOG.debug("Emitted tuple {} for record [{}]", tuple, record); } } @@ -135,7 +161,6 @@ private long seek(TopicPartition tp, KafkaTridentSpoutBatchMetadata lastBa if (lastBatchMeta != null) { kafkaConsumer.seek(tp, lastBatchMeta.getLastOffset() + 1); // seek next offset after last offset from previous batch LOG.debug("Seeking fetch offset to next offset after last offset from previous batch"); - } else { LOG.debug("Seeking fetch offset from firstPollOffsetStrategy and last commit to Kafka"); final OffsetAndMetadata committedOffset = kafkaConsumer.committed(tp); @@ -161,31 +186,86 @@ private long seek(TopicPartition tp, KafkaTridentSpoutBatchMetadata lastBa return fetchOffset; } - // returns paused topic partitions + // returns paused topic-partitions. private Collection pauseTopicPartitions(TopicPartition excludedTp) { - final Set pausedTopicPartitions = new HashSet<>(kafkaConsumer.assignment()); - LOG.debug("Currently assigned topic partitions [{}]", pausedTopicPartitions); + final Set pausedTopicPartitions = new HashSet<>(kafkaConsumer.assignment()); + LOG.debug("Currently assigned topic-partitions {}", pausedTopicPartitions); pausedTopicPartitions.remove(excludedTp); kafkaConsumer.pause(pausedTopicPartitions); - LOG.trace("Paused topic partitions [{}]", pausedTopicPartitions); + LOG.debug("Paused topic-partitions {}", pausedTopicPartitions); return pausedTopicPartitions; } @Override public void refreshPartitions(List partitionResponsibilities) { - LOG.debug("Refreshing topic partitions [{}]", partitionResponsibilities); + LOG.trace("Refreshing of topic-partitions handled by Kafka. " + + "No action taken by this method for topic partitions {}", partitionResponsibilities); } + /** + * Computes ordered list of topic-partitions for this task taking into consideration that topic-partitions + * for this task must be assigned to the Kafka consumer running on this task. + * @param allPartitionInfo list of all partitions as returned by {@link KafkaTridentSpoutOpaqueCoordinator} + * @return ordered list of topic partitions for this task + */ @Override - public List getOrderedPartitions(List allPartitionInfo) { - final List topicPartitionsTrident = new ArrayList<>(allPartitionInfo == null ? 0 : allPartitionInfo.size()); - if (allPartitionInfo != null) { - for (TopicPartition topicPartition : allPartitionInfo) { - topicPartitionsTrident.add(new KafkaTridentSpoutTopicPartition(topicPartition)); + public List getOrderedPartitions(final List allPartitionInfo) { + final int numTopicPartitions = allPartitionInfo == null ? 0 : allPartitionInfo.size(); + final int taskIndex = topologyContext.getThisTaskIndex(); + final int numTasks = topologyContext.getComponentTasks(topologyContext.getThisComponentId()).size(); + + LOG.debug("Computing task ordered list of topic-partitions from all partitions list {}, " + + "for task with index [{}] of total tasks [{}] ", allPartitionInfo, taskIndex, numTasks); + + final Set assignment = kafkaConsumer.assignment(); + LOG.debug("Consumer [{}] has assigned topic-partitions {}", kafkaConsumer, assignment); + + List taskOrderedTps = new ArrayList<>(numTopicPartitions); + + if (numTopicPartitions > 0) { + final KafkaTridentSpoutTopicPartition[] tps = new KafkaTridentSpoutTopicPartition[numTopicPartitions]; + int tpTaskComputedIdx = taskIndex; + /* + * Put this task's Kafka consumer assigned topic-partitions in the right index locations such + * that distribution by OpaquePartitionedTridentSpoutExecutor can be done correctly. This algorithm + * does the distribution in exactly the same way as the one used in OpaquePartitionedTridentSpoutExecutor + */ + for (TopicPartition assignedTp : assignment) { + if (tpTaskComputedIdx >= numTopicPartitions) { + LOG.warn("Ignoring attempt to add consumer [{}] assigned topic-partition [{}] to index [{}], " + + "out of bounds [{}]. ", kafkaConsumer, assignedTp, tpTaskComputedIdx, numTopicPartitions); + break; + } + tps[tpTaskComputedIdx] = new KafkaTridentSpoutTopicPartition(assignedTp); + LOG.debug("Added consumer assigned topic-partition [{}] to position [{}] for task with index [{}]", + assignedTp, tpTaskComputedIdx, taskIndex); + tpTaskComputedIdx += numTasks; + } + + // Put topic-partitions assigned to consumer instances running in different tasks in the empty slots + int i = 0; + for (TopicPartition tp : allPartitionInfo) { + /* + * Topic-partition not assigned to the Kafka consumer associated with this emitter task, hence not yet + * added to the list of task ordered partitions. To be processed next. + */ + if (!assignment.contains(tp)) { + for (; i < numTopicPartitions; i++) { + if (tps[i] == null) { // find empty slot to put the topic-partition + tps[i] = new KafkaTridentSpoutTopicPartition(tp); + LOG.debug("Added to position [{}] topic-partition [{}], which is assigned to a consumer " + + "running on a task other than task with index [{}] ", i, tp, taskIndex); + i++; + break; + } + } + } } + taskOrderedTps = Arrays.asList(tps); } - LOG.debug("OrderedPartitions = {}", topicPartitionsTrident); - return topicPartitionsTrident; + LOG.debug("Returning ordered list of topic-partitions {} for task with index [{}], of total tasks [{}] ", + taskOrderedTps, taskIndex, numTasks); + return taskOrderedTps; } @Override @@ -196,8 +276,8 @@ public void close() { @Override public String toString() { - return "KafkaTridentSpoutEmitter{" + - ", kafkaManager=" + kafkaManager + + return super.toString() + + "{kafkaManager=" + kafkaManager + '}'; } } diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutManager.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutManager.java index 4b60f33d0a0..4054b49f440 100644 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutManager.java +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutManager.java @@ -18,22 +18,20 @@ package org.apache.storm.kafka.spout.trident; -import java.io.Serializable; -import java.util.Collection; -import java.util.Set; -import java.util.concurrent.TimeUnit; - import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.TopicPartition; import org.apache.storm.kafka.spout.KafkaSpoutConfig; import org.apache.storm.kafka.spout.RecordTranslator; -import org.apache.storm.kafka.spout.internal.Timer; import org.apache.storm.task.TopologyContext; import org.apache.storm.tuple.Fields; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.Serializable; +import java.util.Collection; +import java.util.Set; + public class KafkaTridentSpoutManager implements Serializable { private static final Logger LOG = LoggerFactory.getLogger(KafkaTridentSpoutManager.class); @@ -43,50 +41,20 @@ public class KafkaTridentSpoutManager implements Serializable { // Bookkeeping private final KafkaSpoutConfig kafkaSpoutConfig; // Declare some KafkaSpoutConfig references for convenience - private final Fields fields; + private Fields fields; public KafkaTridentSpoutManager(KafkaSpoutConfig kafkaSpoutConfig) { this.kafkaSpoutConfig = kafkaSpoutConfig; - RecordTranslator translator = kafkaSpoutConfig.getTranslator(); - Fields fields = null; - for (String stream: translator.streams()) { - if (fields == null) { - fields = translator.getFieldsFor(stream); - } else { - if (!fields.equals(translator.getFieldsFor(stream))) { - throw new IllegalArgumentException("Trident Spouts do not support multiple output Fields"); - } - } - } - this.fields = fields; + this.fields = getFields(); LOG.debug("Created {}", this); } - void subscribeKafkaConsumer(TopologyContext context) { + KafkaConsumer createAndSubscribeKafkaConsumer(TopologyContext context) { kafkaConsumer = new KafkaConsumer<>(kafkaSpoutConfig.getKafkaProps(), kafkaSpoutConfig.getKeyDeserializer(), kafkaSpoutConfig.getValueDeserializer()); kafkaSpoutConfig.getSubscription().subscribe(kafkaConsumer, new KafkaSpoutConsumerRebalanceListener(), context); - - // Initial poll to get the consumer registration process going. - // KafkaSpoutConsumerRebalanceListener will be called following this poll, upon partition registration - kafkaConsumer.poll(0); - } - - private class KafkaSpoutConsumerRebalanceListener implements ConsumerRebalanceListener { - @Override - public void onPartitionsRevoked(Collection partitions) { - LOG.info("Partitions revoked. [consumer-group={}, consumer={}, topic-partitions={}]", - kafkaSpoutConfig.getConsumerGroupId(), kafkaConsumer, partitions); - KafkaTridentSpoutTopicPartitionRegistry.INSTANCE.removeAll(partitions); - } - - @Override - public void onPartitionsAssigned(Collection partitions) { - KafkaTridentSpoutTopicPartitionRegistry.INSTANCE.addAll(partitions); - LOG.info("Partitions reassignment. [consumer-group={}, consumer={}, topic-partitions={}]", - kafkaSpoutConfig.getConsumerGroupId(), kafkaConsumer, partitions); - } + return kafkaConsumer; } KafkaConsumer getKafkaConsumer() { @@ -96,8 +64,23 @@ KafkaConsumer getKafkaConsumer() { Set getTopicPartitions() { return KafkaTridentSpoutTopicPartitionRegistry.INSTANCE.getTopicPartitions(); } - + Fields getFields() { + if (fields == null) { + RecordTranslator translator = kafkaSpoutConfig.getTranslator(); + Fields fs = null; + for (String stream : translator.streams()) { + if (fs == null) { + fs = translator.getFieldsFor(stream); + } else { + if (!fs.equals(translator.getFieldsFor(stream))) { + throw new IllegalArgumentException("Trident Spouts do not support multiple output Fields"); + } + } + } + fields = fs; + } + LOG.debug("OutputFields = {}", fields); return fields; } @@ -107,9 +90,25 @@ KafkaSpoutConfig getKafkaSpoutConfig() { @Override public String toString() { - return "KafkaTridentSpoutManager{" + - "kafkaConsumer=" + kafkaConsumer + + return super.toString() + + "{kafkaConsumer=" + kafkaConsumer + ", kafkaSpoutConfig=" + kafkaSpoutConfig + '}'; } + + private class KafkaSpoutConsumerRebalanceListener implements ConsumerRebalanceListener { + @Override + public void onPartitionsRevoked(Collection partitions) { + LOG.info("Partitions revoked. [consumer-group={}, consumer={}, topic-partitions={}]", + kafkaSpoutConfig.getConsumerGroupId(), kafkaConsumer, partitions); + KafkaTridentSpoutTopicPartitionRegistry.INSTANCE.removeAll(partitions); + } + + @Override + public void onPartitionsAssigned(Collection partitions) { + KafkaTridentSpoutTopicPartitionRegistry.INSTANCE.addAll(partitions); + LOG.info("Partitions reassignment. [consumer-group={}, consumer={}, topic-partitions={}]", + kafkaSpoutConfig.getConsumerGroupId(), kafkaConsumer, partitions); + } + } } diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutOpaque.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutOpaque.java index 5c5856c215f..18d37d9c27f 100644 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutOpaque.java +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutOpaque.java @@ -34,7 +34,7 @@ public class KafkaTridentSpoutOpaque implements IOpaquePartitionedTridentSp private static final Logger LOG = LoggerFactory.getLogger(KafkaTridentSpoutOpaque.class); - private KafkaTridentSpoutManager kafkaManager; + private final KafkaTridentSpoutManager kafkaManager; private KafkaTridentSpoutEmitter kafkaTridentSpoutEmitter; private KafkaTridentSpoutOpaqueCoordinator coordinator; @@ -50,19 +50,12 @@ public KafkaTridentSpoutOpaque(KafkaTridentSpoutManager kafkaManager) { @Override public Emitter, KafkaTridentSpoutTopicPartition, KafkaTridentSpoutBatchMetadata> getEmitter(Map conf, TopologyContext context) { - // Instance is created on first call rather than in constructor to avoid NotSerializableException caused by KafkaConsumer - if (kafkaTridentSpoutEmitter == null) { - kafkaTridentSpoutEmitter = new KafkaTridentSpoutEmitter<>(kafkaManager, context); - } - return kafkaTridentSpoutEmitter; + return new KafkaTridentSpoutEmitter<>(kafkaManager, context); } @Override public Coordinator> getCoordinator(Map conf, TopologyContext context) { - if (coordinator == null) { - coordinator = new KafkaTridentSpoutOpaqueCoordinator<>(kafkaManager); - } - return coordinator; + return new KafkaTridentSpoutOpaqueCoordinator<>(kafkaManager); } @Override @@ -79,8 +72,8 @@ public Fields getOutputFields() { @Override public String toString() { - return "KafkaTridentSpoutOpaque{" + - "kafkaManager=" + kafkaManager + + return super.toString() + + "{kafkaManager=" + kafkaManager + ", kafkaTridentSpoutEmitter=" + kafkaTridentSpoutEmitter + ", coordinator=" + coordinator + '}'; diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutOpaqueCoordinator.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutOpaqueCoordinator.java index 6e85735e76b..7898b6e8e69 100644 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutOpaqueCoordinator.java +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutOpaqueCoordinator.java @@ -57,8 +57,8 @@ public void close() { @Override public String toString() { - return "KafkaTridentSpoutOpaqueCoordinator{" + - "kafkaManager=" + kafkaManager + + return super.toString() + + "{kafkaManager=" + kafkaManager + '}'; } } diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutTopicPartition.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutTopicPartition.java index ba6126c9e3e..b020bea4367 100644 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutTopicPartition.java +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutTopicPartition.java @@ -43,7 +43,7 @@ public TopicPartition getTopicPartition() { @Override public String getId() { - return topicPartition.topic() + "/" + topicPartition.partition(); + return topicPartition.topic() + "@" + topicPartition.partition(); } @Override diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutTopicPartitionRegistry.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutTopicPartitionRegistry.java index ee5220ee401..2d50ca78e7d 100644 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutTopicPartitionRegistry.java +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/trident/KafkaTridentSpoutTopicPartitionRegistry.java @@ -22,7 +22,7 @@ import java.util.Collection; import java.util.Collections; -import java.util.HashSet; +import java.util.LinkedHashSet; import java.util.Set; public enum KafkaTridentSpoutTopicPartitionRegistry { @@ -31,7 +31,7 @@ public enum KafkaTridentSpoutTopicPartitionRegistry { private Set topicPartitions; KafkaTridentSpoutTopicPartitionRegistry() { - this.topicPartitions = new HashSet<>(); + this.topicPartitions = new LinkedHashSet<>(); } public Set getTopicPartitions() { From 3e232e2fc70c8dcc1fa1f79e0e2b50dce37072b0 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Wed, 8 Feb 2017 11:11:24 +0900 Subject: [PATCH 19/29] STORM-2281: CHANGELOG --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index e705c9349cd..c7fb4c84158 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -197,6 +197,7 @@ * STORM-1769: Added a test to check local nimbus with notifier plugin ## 1.1.0 + * STORM-2281: Running Multiple Kafka Spouts (Trident) Throws Illegal State Exception * STORM-2296: Kafka spout no dup on leader changes * STORM-2014: New Kafka spout duplicates checking if failed messages have reached max retries * STORM-1443: [Storm SQL] Support customizing parallelism in StormSQL From 677971d876f9ff35801cdaaf81a025f592c72059 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Wed, 8 Feb 2017 13:59:39 +0900 Subject: [PATCH 20/29] Convert NoNodeException to KeyNotFoundException in getNimbodesWithLatestSequenceNumberOfBlob * since callers are able to handle KeyNotFoundException but not NoNodeException --- .../org/apache/storm/blobstore/BlobStoreUtils.java | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/storm-core/src/jvm/org/apache/storm/blobstore/BlobStoreUtils.java b/storm-core/src/jvm/org/apache/storm/blobstore/BlobStoreUtils.java index 6486ba212e5..b67c2c610e7 100644 --- a/storm-core/src/jvm/org/apache/storm/blobstore/BlobStoreUtils.java +++ b/storm-core/src/jvm/org/apache/storm/blobstore/BlobStoreUtils.java @@ -30,6 +30,7 @@ import org.apache.storm.utils.ZookeeperAuthInfo; import org.apache.curator.framework.CuratorFramework; import org.apache.thrift.transport.TTransportException; +import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.NoNodeException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -76,7 +77,15 @@ public static BlobKeySequenceInfo normalizeNimbusHostPortSequenceNumberInfo(Stri // Check for latest sequence number of a key inside zookeeper and return nimbodes containing the latest sequence number public static Set getNimbodesWithLatestSequenceNumberOfBlob(CuratorFramework zkClient, String key) throws Exception { - List stateInfoList = zkClient.getChildren().forPath("/blobstore/" + key); + List stateInfoList; + try { + stateInfoList = zkClient.getChildren().forPath("/blobstore/" + key); + } catch (KeeperException.NoNodeException e) { + // there's a race condition with a delete: blobstore + // this should be thrown to the caller to indicate that the key is invalid now + throw new KeyNotFoundException(key); + } + Set nimbusInfoSet = new HashSet(); int latestSeqNumber = getLatestSequenceNumber(stateInfoList); LOG.debug("getNimbodesWithLatestSequenceNumberOfBlob stateInfo {} version {}", stateInfoList, latestSeqNumber); From ec5c9627c41babd1b404952608fabc99ed36b9cd Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Wed, 8 Feb 2017 14:31:54 +0900 Subject: [PATCH 21/29] Fix RAT issue on JoinProcessorTest --- .../streams/processors/JoinProcessorTest.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/storm-core/test/jvm/org/apache/storm/streams/processors/JoinProcessorTest.java b/storm-core/test/jvm/org/apache/storm/streams/processors/JoinProcessorTest.java index a8ace8ac8ca..08a6c7da037 100644 --- a/storm-core/test/jvm/org/apache/storm/streams/processors/JoinProcessorTest.java +++ b/storm-core/test/jvm/org/apache/storm/streams/processors/JoinProcessorTest.java @@ -1,3 +1,20 @@ +/** + * 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.storm.streams.processors; import org.apache.storm.streams.Pair; From 6c805c6562c3c98deb584cb1c9fb3435abd0692b Mon Sep 17 00:00:00 2001 From: Tibor Kiss Date: Tue, 7 Feb 2017 06:11:32 +0100 Subject: [PATCH 22/29] STORM-2350: Storm-HDFS's listFilesByModificationTime is broken --- .../java/org/apache/storm/hdfs/common/ModifTimeComparator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/common/ModifTimeComparator.java b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/common/ModifTimeComparator.java index de5613eba51..0558b3fc456 100644 --- a/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/common/ModifTimeComparator.java +++ b/external/storm-hdfs/src/main/java/org/apache/storm/hdfs/common/ModifTimeComparator.java @@ -27,6 +27,6 @@ public class ModifTimeComparator implements Comparator { @Override public int compare(FileStatus o1, FileStatus o2) { - return new Long(o1.getModificationTime()).compareTo( o1.getModificationTime() ); + return new Long(o1.getModificationTime()).compareTo( o2.getModificationTime() ); } } From 3985bb84037accee772844aa299caf35440293ec Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Wed, 8 Feb 2017 16:29:02 -0500 Subject: [PATCH 23/29] add STORM-2350 to changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index bb08be16d81..c699f6ade30 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ## 2.0.0 + * STORM-2350: Storm-HDFS's listFilesByModificationTime is broken * STORM-1281: LocalCluster, testing4j and testing.clj to java * STORM-2226: Fix kafka spout offset lag ui for kerberized kafka * STORM-1276: line for line translation of nimbus to java From d6c3145b6c64c23c58fecce24dd15d02670cebf2 Mon Sep 17 00:00:00 2001 From: ambud Date: Sat, 4 Feb 2017 13:32:16 -0800 Subject: [PATCH 24/29] STORM-2344 Adding Flux File Viewer to Nimbus UI Adding apache license and link to Storm Homepage Adding links from storm nimbus homepage Adding License for Javascript libraries. Using min js for esprima Adding license files --- LICENSE | 102 +++++++++- storm-core/src/ui/public/css/style.css | 7 + storm-core/src/ui/public/flux.html | 157 ++++++++++++++ storm-core/src/ui/public/images/bolt.png | Bin 0 -> 6019 bytes storm-core/src/ui/public/images/flux.png | Bin 0 -> 5328 bytes storm-core/src/ui/public/images/spout.png | Bin 0 -> 5255 bytes storm-core/src/ui/public/index.html | 7 +- .../src/ui/public/js/cytoscape-dagre.js | 192 ++++++++++++++++++ storm-core/src/ui/public/js/cytoscape.min.js | 63 ++++++ storm-core/src/ui/public/js/dagre.min.js | 6 + storm-core/src/ui/public/js/esprima.min.js | 2 + storm-core/src/ui/public/js/js-yaml.min.js | 3 + 12 files changed, 537 insertions(+), 2 deletions(-) create mode 100644 storm-core/src/ui/public/flux.html create mode 100644 storm-core/src/ui/public/images/bolt.png create mode 100644 storm-core/src/ui/public/images/flux.png create mode 100644 storm-core/src/ui/public/images/spout.png create mode 100644 storm-core/src/ui/public/js/cytoscape-dagre.js create mode 100644 storm-core/src/ui/public/js/cytoscape.min.js create mode 100644 storm-core/src/ui/public/js/dagre.min.js create mode 100644 storm-core/src/ui/public/js/esprima.min.js create mode 100644 storm-core/src/ui/public/js/js-yaml.min.js diff --git a/LICENSE b/LICENSE index f8ee6e66cdf..8569bf7e236 100644 --- a/LICENSE +++ b/LICENSE @@ -283,6 +283,106 @@ THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS "AS IS" AND ANY EXPRESS OR IM ----------------------------------------------------------------------- +For js-yaml.min.js (storm-core/src/ui/public/js/) + +(The MIT License) + +Copyright (C) 2011-2015 by Vitaly Puzrin + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. + +----------------------------------------------------------------------- + +For dagre.min.js (storm-core/src/ui/public/js/) + +Copyright (c) 2012-2014 Chris Pettitt + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. + +----------------------------------------------------------------------- + +For cytoscape.min.js and cytoscape-dagre.js (storm-core/src/ui/public/js/) + +Copyright (c) 2016 The Cytoscape Consortium + +Permission is hereby granted, free of charge, to any person obtaining a copy of +this software and associated documentation files (the “Software”), to deal in +the Software without restriction, including without limitation the rights to +use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies +of the Software, and to permit persons to whom the Software is furnished to do +so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED “AS IS”, WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. + +----------------------------------------------------------------------- + +For esprima.js (storm-core/src/ui/public/js/) + +Copyright JS Foundation and other contributors, https://js.foundation/ + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL BE LIABLE FOR ANY +DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND +ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF +THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +----------------------------------------------------------------------- + For arbor.js and arbor-graphics.js (storm-core/src/ui/public/js/) Copyright (c) 2011 Samizdat Drafting Co. @@ -586,4 +686,4 @@ THE SOFTWARE. This product bundles PMML Sample Files, which are available under a -"3-clause BSD" license. For details, see http://dmg.org/documents/dmg-pmml-license-2016.pdf. \ No newline at end of file +"3-clause BSD" license. For details, see http://dmg.org/documents/dmg-pmml-license-2016.pdf. diff --git a/storm-core/src/ui/public/css/style.css b/storm-core/src/ui/public/css/style.css index c4c41fdb68f..f52ad414cab 100644 --- a/storm-core/src/ui/public/css/style.css +++ b/storm-core/src/ui/public/css/style.css @@ -19,6 +19,13 @@ display: none; } +#ui-flux img { + width: 50px; + padding: 0.5em; + padding-top: 1em; + float:right; +} + #ui-user { text-align: right; padding: 0.5em; diff --git a/storm-core/src/ui/public/flux.html b/storm-core/src/ui/public/flux.html new file mode 100644 index 00000000000..25ff787d473 --- /dev/null +++ b/storm-core/src/ui/public/flux.html @@ -0,0 +1,157 @@ + + + + + Storm Flux Viewer + + + + + + + + + + + +

Storm UI

+

Flux Topology Viewer

+
+ +
+ +
+
+ + + + + diff --git a/storm-core/src/ui/public/images/bolt.png b/storm-core/src/ui/public/images/bolt.png new file mode 100644 index 0000000000000000000000000000000000000000..06b58937d51173224618fc89ea64363dc34ac74a GIT binary patch literal 6019 zcma)=1z1z<`^V|-8n1LHBEqOKMvP7gX#|PUx#2dtTTn1)X(S~jB_t#Tqy$8AAdSEz z1rdJeTmG*1{l~eko%5XM+~3a~Ph96L=C+O+CD|1+92^`<4RvMx%d5rZ&z^+n@(eWH z#lO7ZqV&}iaVmybH*jzW{G1I<(Wcs3Qnm(7>ayu>kA`vj zxVynoQa-ZWKP{v#+dsO2+?+p6(5|xFrrNhTl@LgKPLPnekT5rtjFXd725ILYrLU~| zhy3zRmfH!9_K*Ssy}iAKyv2kNNJpRu1OfpHivmSO1urcGQND0A%tsK8;`tr&Cr8;H zWs7w7KszJgoIkiQ8-yoXmYe&BqrZ+n;&k`W*8a;C9Q8+Dml*-mbAz`4qJMiC> zD73!)-$DMX5@qP?VGq=|M(!g#=tDEl8l%5wi1<*%wA!k=kMIm1zC7~IxgLm7HW5^{F7 zlM)t@6c!NyiwlEAg(W0KF7G78loW*_!lII3MM-648Q?#p|5g7#$aV-@Pj`Db`VX?* zC0SJxtomPMQBeg+QAtTfVHF8+Q8CFs$-hYdCd&Z-k?PM0_WQX1S%-f*g(#_lRF%OJ zqGHMtA|fgvh={O=B1B116{ILG0s?{l;{GlD#p&-pHU3$pKd_+x3;T=oA6Q+av!gQ{ z=BE6&>mP&g|0~t+k^Kwe7dJoVnbf7pH3Tm!v zjmJY7s3U2vl)=o5M(@@6w8v3{)UQxOC+uryYw5cg`|rT`ikWJBc$p+~dOLBxP130Kv4vIwA!E2C z9Xneg8hK8?5?GV8lGwJ|c+$x6*-V^C_Nz5LR_A1Ptj_$F<6W)MMrJzjZCZ*=Fq}A< zRJJ52(8W8#Z|P~jiP)ld|9c@pS;Y7Gb1ke5L4(w%&#pSe36yih>9L*n37AN(K-1z_ z`C7K=&~8p(51|UY;aFxha6ErG^QxWj#}V|3jp*HF+A!>K^^;E#4RfKEEzaJlzqv61 z?$cq*qG-Zja>2bsQ_Gx`&)}F!C4vujS9@_$&G@f(?po&!eOd{6Xn3t-EU8(3Vx+)_zAF0Uso6a5^8{kTK;?rqSL;0QBg`kux7t^W=0dU> z&OJE$Kh9C)hSYUxPYQKS9CAy3m`-+2^9W*L3g8_4NJ4+6SHG8mZD1L>BkmX8G4-yV zEzI8vJk|2Dpz;VMD$FNty#i5RuNi(H_L}@WhQ8I!*{wIV+!}m_?8d`yB@XZ%iPvq) zkOVwueiS?q-{USz>m(66G3pDY1!Qn|3-HjNdg1yh**e|$ET3R}(<@wrA_%egEbyKD zN4M)>=C_dwYQffi#l#~j*(03Y_Z2}2>1-FyAS6_wl@XJ<1@+CUPVsj^ww^u zV}X{Oz4=2J)x*f@M|^;}vFDUdFS$liGQ5<>I;76y&&Kv%rY-HwPp2zI)ZA%DOjG8( z5Ln%n%Pi3X2k7)}sutQ#pVh4>d4>bM21m3=azoh8F7|mv=IdvO^W~>Zv%|t3r#n!cqlVK?n>K{Jhe)VyOF1@C;RGWmbY1i zns{x9f#rMQnNNJn3hQ1K4{_jtN3|z#tCpSaKLrs306SE-Tl6dmwz$gYFlT=F8{}o~ zCH-B(cGjg6L=VaRZZc-4xW5qb=eOc_oo$MBzBg2)fhi*f;5jFLrpO+%N_2-QbG#=x zFLm+ck>+r;@jXA?%Qc3)Ku)|TtmKWS z*FS%oInc`Ra9mKBx!=GXqNot(26hj>r6cdJLzGZ75 zT}Dd{f^3I!#0^`+@aOpoA6J#^)59#Q;0qzII#SvSlN9F|O z0imAqOeP&CHx`p)y@MQ=cqmVpjyrP(G_@ZPU56_8RN02Huh_(NbS2%u+Oc)}c%D2} zEI_b5^ThU|lg@fI>y^}K%MT08nG}mj7&)(0(=+V)9v%#|>4ZGq1Vn%3xzf{H?!Z5ArGVHk z>mi5eel~?NoKWE6Y2%QeP^weST=Dp}^m^xHEAyCVD{~gj_3{iGhFv;}#GI#e0`bOf z&-#j^8*iaDAO^Z>7LtWft^&=p^ojm5_I;Fh3m9QjFQwBAVix zbmw!+w{!@((&acG>e0XZ4(TERh~u6OX*X`d&)#^n8KB!NeEddkgo|3_MU6eCxa3Z* z4C2`VNd={K_5Ac|?b#T0MJctuA=y??mGPI)x!s?hHhht4BQKK)>?cw+@_`lX7*+z% zt)bAJx3O<&0Q^-3IeN987f;XR?8WZm*8$svt4!>^hp=1Jr>z(w(!>cw>l63yr|D#M zQhE=Xt$9Pb%d#3>(mHLr^1MTUyA?>mDjNtR1;M0UMQ$3sv5YrcW-Gd<@5#NInr#3}r6x%Y+VU(BedA-qAPZ;F# zki=mqkX5FbZ}tHT-iac}%U6!61SZ3~PPQ=({-z*JD;?4oql#ZQD0-hH$I%{XtGis} z-F+w}jnxLR^jHzHTrYR7)+6{N&lG)7xid1pbv8squW((ZDk+kbP{|LZK%qE5(64_# z1XUOf-Tw0IE^k@&Q739dP0348=Kj}uMr?#l+{nUC8*1qyH!|kRZjdXO39TkWG@x8W zOcg57bc|QG?HSgrD*q;1#(F847%p@zb3Xz{Lx~0<=^O3V!0t6=y1|-K7b3wZp=5Pi zq+N9NrkirTqUrG4*Ohcnny6eGLXrX@Aier>I@9Jh!D>QE_}fhLFIu;>2htjHz^^N~ z;Ysi=KXoe|eyER6O!HikQM^Ut2EW{cMxC{)gcKv$LcZ(QsAt|{84g|+tbAy1@-4fZ z&xWAbB$N3H6nxWlv$QpykYz2Z7i>mn8u)QmbgHU^ni1-vNn96}pg6@@-~S>hnR*y1 zM@4_of-h2`k)LGH`HG&|Jd))u`9LhLgiAznu<^q8mxeYn_<^TK+lizSq_gTa63(OK z6@=_inTFkpJTbwf`OJx|QvML@Q{G)(ha047Pnga=EZ*|&(bvFI9LQx~8K=H>Btq8L z(N9ZF)Hh_v_O@0#k+aWd(lN}^L_;lYvt}*9n0@~7`>SC0u5q!E&J60%wa1<1c_fJ~?tHU)MjlX7xS3BsFvX=j%p7S; zGIO6N{2N;(lCuXy>NrZ+KK>r`yq6+_C`bw;{ViUacI&rwOut%ja_rb#dp39_+@6mO zskFaQxD=FX;U@f{Plf8UWeB)kkF5Sw zgngykybxoqvHoRntPul|cr$c`l}-2h!DfTststwRxS5uxDQT=#$QQCt#2P+>z-L)^ zq{=$Kty}LHW4NC1%`+zWfisbcw>(AY%K~`azI6oGSr!yN9wpg-NCl;C$-R$*>hJaC zUzTrq&>BfuEx{P~;+@#YN`SKwXJ7!%8}BeoSB;i{hN^wqfWClwgn_$shncLysq<7^ z){;l$Z&!&rxbUk{TOHp6$Z!?m)^YgdR8hGJJ_ih4`<^%tl>#>J^%((&is?m@iz78p zhdyrkg<^hNw{|U;AQ;<(*@;(d&F?VmmWd<1w5&FI9+`zEZGB6?=UX=0%1r7g$F#@K zLi~fb9zxUl#YP}>e7MW7w@V$YkJs@o?!wE53n}Ia;AIC}i-uAzM1c5$sChZYICqvp z5NmXh%S={b$7b03$1~v0`YPi_JLO{Oyy@HiLl5375`>olvqn!Xt=+ZTPWl=bcDd(`9@zq0Ac7X z-nDBV3l>J(-gnHne9zfTE3hqNtEMNqgWDZ>p%)ZM%Cevnt{R|e%X1?*N>@LTX zf3yE2&><4Z9o0=PGlufFW}|gisozbXURQmSdo-( z8)9OeH<(HZ3^~NrZ+u49ckYzROPD*9qaN(Gj~=j$ob@G=ILK;+kH%&Ds(J4eET2Qm8B2*P~uBCPx93C8(?R6bUortv7euv#p#fZu!S=^j7Tz zUKh;7#~YTfyzBEip7umG;$8yswjD?Q(`&A4FX`?aeYzgdQs(`7$T-xVZySIUoc82f z&{LiHgD`o?5+TpfdvWIh?PsyJC2uq8{D_(i0UNPF#NQIU%Y{_yHB_u4AMCNufUS~c zWvz|fghx8DY`ynZao+~D3xL#~KMRu5$d>xDA;bcDNt;JZuSFH%IC?RWaF~1kv%*kB>nOWRuRX2iNIjm7`-9T zqN=**>)^f&S-mxx68>$h#=E3EqX_O9yelW+0jCJre9^v`jmeyvLtFN_W-aJa&Dbmc zVBhWe46)aBhjI5lu`_0mM}yGgvy*XKfn%@JV1;YESmm7-lcV&lYT08}^u`QqIGAI0 za^eOiU3!+XRloLhnD129|NAuAvF{CxPKAH%XQrN!3*o}^R{5GN=9`c**a~fxW8Xaj z)@#9G8Yh5wCkvW7t$m@?jK@0H1+%~V4n7VN&WxAR?5pJbzTWmqVcvJ2m~fLIu#bFj z>-&TP5uGoM;y0mL5d($#@=^6*Dh^K{K4y|&%CTZO&4WfS+jFG4FA9&BU^5K$Z3wuK zfjQg+vvJ&dx^rJ=^B2n%xKEbeNOTS9q8E??A3 zVi!uABycUfj_R5pWKk1#Yts3u7HwW^!{fngFbe1W_j!wsBqY-8UvvOU2L9aBcHxFl;R*+#Cfk9Mr5hZbj?K>r(I_EV}E69Kr1o>C~>L zqaL_X*QN*CNFV3c(0u3!4y7*Th?+VR^4VQ{R5=p>>Kp_PyM6#1s_`D-t?TcS3zsb6 znk7m%b3=4&8J87K^1`y#+h+TNC+R7<#XSgdWKzk*CHs9$;Vvu5pyEXoRuY?vVK`US zZFw2c&qr!m)MmsI+v6;APCP{~V3|VP`&=dYAk#cqbY`-o0*x*?GncX)o=#s_yPzTH zZ);_>wx~kB#LXZ$%27S71;#Qe&JeE02Kid52pd$sBb-%vc0?nye>RIMkieU9JG6P; zru|~2kc2LQp_O%(j8Z1y+g!DU``8nQt0@HQ7D{n*iyM+M-*usw@)+uS_n~NL0Z-$* z;jyc@QdF^A)*JZc@?lApeT~ltJ)@G1;%D{>aGQiH@U69M2^+%d*1madUkEgW_$nYa z?*o>mizw6`TcC}O2Vz=%;UDu?t`49~HnBMa;9X3cHS>GC2N^PTy{OeB4tR;{Obr$F z>Bg`t=Zovl=rfnSZuzT;aSO~at=hS>v7_S;wlJ^H=hvbsQ=C31OajBbzj)JAG+CkTYq?JmuZA z|B#?$(5ScDhzKwub1i<>*WYT7yj^SA%k+4!BUpD!#k7{3j36kBR5)l4ZF-9@Cns!||hdxw?K**iooa5$rXd|nVVBn9x%QaMVlq(di!~O#(L%Tiz literal 0 HcmV?d00001 diff --git a/storm-core/src/ui/public/images/flux.png b/storm-core/src/ui/public/images/flux.png new file mode 100644 index 0000000000000000000000000000000000000000..97016b1e920b344f96d3b0d7cba96a55d25fc1c1 GIT binary patch literal 5328 zcmeHL^-~mHwBMz>RHVCGx*L`brMqE?C8TpnX(W~IMHX3>kOo&;KpN>1=>`cw;PL(C zy?^4JnRCzFJ9Ex_=5y!HozEvhPX|PZM}-Ff00`C9lwUv9f~U;D!FpPe4dpUV1^D(g zND+V-quG1f;CiWisn`?|pB-}@^YG9Bl$vSNxoyc{nzk9Fcv|5;;aj=G6kJBfH}V^h*nRaG^9;-ND?85&@Jz)@o%$`N#}a@Y8?)Tb3t#9u%@un ztv4qo-n%U|d(bQAO0+_zb!$_ymsKp?iczH5$Cax~6T`euw542ML$uZO7JnKI{n`X+` z4W`KeX3h-F#Zm!?&-L~75lq}#IO6E;bR=Uc z(y{(#oQXf=wEEO!b*=DA+C;FAq~L#@nCUQg6WP%$OWsNRG%!e^VqLPBnT+^$7ewZ6 zU&|7K&1++0Ulf_$7DL;kcN(_qXMv*h5QVGimU^4;EV-4kWnOQZLIo5}7)Kd$C@9)l z_E-(ME>-oVWs|a^KdF9Y(`?psXQPj)hw67dV5tj2i|f$kUS3-9mNp5GVIjW&t0{LG zKCApfFFo1nYC<`Xi~IZ@!oJx4Aw@{QTS|5U%ZDr`I2U8=@xfn%^ORJo z@2Y>LSmwFpgZCTi!<1ZvlE%@D;dC%{vT^sV927g#LnpgtafQt6|5>hq-}0w>+H|TK z&fK{p#$qC6BK>#xY8?IUTvF_o(QOk1Vv~gdX#)O0uRyhfh396M3Uh(j`894`aIkKP zL&qnoSek56=PKsazwAPfiML}Cxtt50riGRP>^(#ZT9~h3lNy!|p|m7{aXA$I<~Zg` zJLqt@`!~Qkt35QU1zt^TVp&8#V4mvm+tbf)bWWra->8U?UOF}{1l)Nf|B=a^vG@ST zjjFviRu%T{S630CAsZAX$@Yc}bIr(HX)b_WlYI>Yi9Ej@Bb(12Faa;falK^4yv$$P zKP0%7^KsIz5Ek%@SHp4pxI5QvH!-cTa)a?lo|~#YoXBwwl<_TgN_S0vasr&f(xvrm z6>STFkP^|BfTqcC3$un-m8&io>(8CdPV9KM65AvpRZ}DuPZc*(@ITByba~HiRyQA_ z?b9q%6FO6%H!|!W`xx*1a@loQOyae98)W1PUrkO6TTaosX>EM*`47k2;O@{@j#?3z zNEMUvqP^L^yeo!2VSIs({(7O!eYda3mqGJd=Q^=a-cO01qUyyn4xjiUpsHtM`1GC> zZ$1rIy%GM>^||{#$@X<__=`Sr+mzshNXbIc&X3#Tj2r58W!zeKG^dzOWnJDSpAj-9 zm%1PGL(jzUZIa(+!m86OU5fr4{p}0bcONYe`TT{}Sf^lMIiJMcQWc7y{C(n>>+K;M zOmf#NFywaaODTuScs%PTA)93D>{rI7%N$Q2G!25VP~Mzl>cNB_Z6&(&?SU zbAs8al?e3W1Qu_>3e(PJ5gGoD63njP?FDqpNoac+0UZFUg)%RqE^%S?R^VZ)s)2}fuyWUf=1dCKV=V$vtl73-q_rA3A@km z#L0TPyf->}x#X*Mo32)T<dIgNb(20WS@yx4z!2c1v-2x>F?g95URtW(4t&dYlh{;K5o4U8<|wA zR0-aWhObtKi(Jd00Q8;nkxuAS*;&Q{cj1IJFpdQ)dP*wJ`3|tR7qrw z_W_ENO-~&=qp>}v2d!n|nF@MwjVauCJ?ll+7u5A<4j&%2+i%P?Y$MXI=x17OH zE+Mj`HXse|l7ZpB*0pf~UVE41*f_tBzijEe)B2!iI&W6jP0P%hjD&R>O)nHh7DIpC zECsZ-ERHpv3u=zPSXaZ%^3|c(W^Ba6+*>axHfoiQ?4tVSJbjdyw#}zKR|GPSPwlZI z)D^v^?>?BNU8A;j>{A@4U02I>3Ykm;NJ<`#h04FG;$Do%k&?>EcNX-^GBTdy3Kbs5 z$cYk41e!xU;5?ngArX{IL&;Il0!>u98DNEYGCeR+Ma73aM?@@JtVt-XuW^W!msgZ( zLBud9^bpbP@)u+3VEAiq4z*WWch3wq1PH>KN(=Yp{~V(Zz5puW?2)fD5Z~E&3XZdS zXT3lmuo)YL$R&P7uS_`bhcb|N+Vw{}qc|ZgKdP?7e8mXn1}c#7aJ)Mkb8gp;dC+z) z2Cn93yQ)Rq39GvnWHnPz!Uk(hglqFlND503)3g_%xFE*QDZ=8D$KN{8RIoASqLR?% z^R6(J=yyRevOkH+z!JVBMKQ_OgV1XnqNcASmWKZv2m<>HFt4G^Da7JAM1L-cS5seA z$RSMtnt|t2zPx+~S3ibT@QG*I8jE7x*Itd-XXzMY`4#(s-Wu9o>GUkdHkEkCLY3#sNs`KWtFt~bgU6| z@SCLNS=M(0-J3_(O=vWf3bA=Dq?Q-n>nbuM@%x-fX`+VPI zo1dWjyl-TGQ7%6Z)}8#Shq|00p96f2v@TvV__BMMKz4c+YKE4Y9PfAi811M%1CtuG zGsyC785`@K((WiSxlvYh^J@;16~Y`zGPtB?Ql}S9LC{d25s@F(@=214(mL~3@vaZ0 zLP9obWQ)5oqI=ylC&j&~JBs{m_t_y=Ci-*UT`;?iiZZMIM4&-=#5pnX>OTwZy}Qqy zWZLv%=rTk8zh;FG zjk@7cXoaBCe=N`Te?32V=$JIWsGld9bsJ3!t5gEKNIG{LWe_+iSIPgfX}@9~Rp{w# z`lmp{j1d6BQqG5@?HeB}T_#3sqieKl6rjG#c43gZBT`j-aTwr}l-sQB^0>&OBPofd zrY={~O&GJ0`uuJzkS4dZ8t%dlBW0HSBT%p)Y#8$Xa^BcH3`9x3c?hch-tB2L7qAD? z{qXgV3Fneixt9j}ZlNBOIlr{x4p$?ZzJ8IIb^0fsb5->onbRXDv9-#6Iz@p|Q| zY*B4Ww?$YHJ@x15^&k2N@@-3N8*hTd8m%OiFyI{lUt#JKD(d#;a;jRp@6j%kRu{jJ zTOk7ci$lXn#=-SK>>B#M`nsy8-@h#QUL^n73?;m*rBqu?l2>KgHuCAVwnE%HtJ75n zEpKXn81|1p(U5Buh)O?0AMCy=w8E+rq`yma74kZC;~@qrPhR0td1&Sx^@jvuY4SK| z;-S(@qXxUnSpIsl_1XpxXTbfoe>(>Y9M&f%yo{f8lG=QeO_t?OXJj=UDGLUG#!Bq1 zv9T{>IrLZUyoqwix5|SH$9tk(fX&2y#bU?r01d;Gnzt2CuaVDxu)CoRNE>tQSN9rf z*dAfGDctoblgR6L(RzIyh2UN;UO=fs2=$}=aA<(`##!}H1lNU@1}gpA-n{W_O^zNl zS2~gW_pA0}DeKdMu3ZPFi!Ii=3QQV_60rS-8+;D*7@@3aWK+_6ZZ3&2VHtL39R!pw z+fs8yVamU(vA1TwC_vEA{JDCm+s_EiOvo9!Rwou06s5gOEGOkcW4`xB%HDJb!XJNU z(V|gcXKOWPL4~whs$4f@50Czd!sq7cz?IzE?5m+}n1r(VR~hm#WL}J8eKgw;UnUm` zJuS50Z*rdd5r2=XWibTUhWp|ZFqSEl>`bTqpZkc|(}5nR>)e6-d_=%xtD75rHB^ed z`6&2j`HJJMu`kQ3Er`tPnLL2oLrnLS>!1DC?(VjR-EXG}8P~NK7Rh00 zDbh=Jqpvyq$-ETXG9w&W)^D!U^~f|(xLZ08+3C>DHISk@2ar8W>aVdXT51G6E^8@9 zf!^_NYf4=4hcK3`C(!@j_RG&K0cPQB*pO~H(spYpp;j+0|o6V#qYY^W-7#_N&e*l8NhKBIb|5o_wO z;^3_mls096@;-+QihNyr3{h(ME3MAq_%sjb4)m&XwW@0w(enTXKO--l$r{#b!z9q+ zWl+FzTg>4#U+ukrWl$@0adD9Zvry!O>X9>Okv8Ml$9zGd_!HY#Dk}T!+CUs0m4a^| zkqodX3VZ+&PAFE!-!o+BO1s|&-L|aP07f2p6Q-v&y~1OrN;GIqZSB!O7(Vnvs!i2P zK{b4&85F7f5v@4qKR2Y0A0XZiB%_CMlYL%j_1-k}<&SQHn&Okh;b@{>!~&Y*=Q!5) z_xG)OPpU|SNa&Ta6wILgXfdS(&|@&VInb&cT5kQxxh8TzsFk-Cfv}=gV|-M@R|W&o zr7B}sdki@KszBd9nHv;1L;%8RQu7)(d^?6sr#Yn1j7opNp>fhA3|A{ zkYYEe>nfj=XkkIutGLbb#2F6CgX7SfHfzR zSoKNlZ^lPvmp|DM6E0Ej^Syzf6KZPeI|Y;EC3*FCq?>C)smk$NzdcZBF}OWg;TRgC zJp{JSOc%*ZaroMuuCwr&<>h728PB|%NB_Oq0s{Zb>+==rA3RjI93o9?IeYis8n-E64H@5aI^wh8s>CRnWY7DuNqe=-BttCSKlkKDjBXo_+>Bz{IH=xEmn>-<2+3 zP`UDipH#E$D@o(i7eR^B%3-%M@;@DUW*l?=JpzlxI?1UKyTB(`5}>Z4ql{3riTWSC CEgs$g literal 0 HcmV?d00001 diff --git a/storm-core/src/ui/public/images/spout.png b/storm-core/src/ui/public/images/spout.png new file mode 100644 index 0000000000000000000000000000000000000000..691f2350de6dd7e2319fae4ebdaa75d35a79838f GIT binary patch literal 5255 zcma)=2UJs8x5p9bNS7*LKx&jU2qh4tcL*J&4J|;>04AX$Dxe4=(m|>^bO8YclwK4F zAWf{U-Z!XGL8!vaA*KOP1P1Adqgt{{ak85#rBy>Ms%93%sR2rDxH001SN ziz~uZL+huUe1#NtClI_5V6dN`AIMJ{p-tpz+Q)j5h(}5SIZP$rW= z7>o-70+okAp>i@1IT%D%9!kEFmsVGUC_rHHa%%D#8cN`QNdKe$Kglj$&OTVQC*ddA zg-q6xm(%(WG7P3F50jTygJ{aiz@+8>B>y7)o2&%>N2)&-*dO!$XC40OR6$(}uB9O- z3zOE6g+evq3Q!1CO+j5v3$7*ug~Q>0asQV7;`9%ny8o=wPb~cZ!~P=u2Ns3HxM4gU zuWJ15`gjulyHfoT*}ouuadW)R5M&ebG&mk_8fB;w_#b1x*przEbsXA};Ds~y^17<5 zdfW>D#_nDOFTA^#H$dG?79e5h=#23^9+3P|y8jUTWGR7Xdr>c4ppsNBpm5ibD-6}8W$ z%ebY9kbqjck-{$WrzLsHY@qLNhFLLQ>@3N(sI;`^?s#N+vpYun;Gq%4$!9t&p&5yLd1yu_Xxuzl0k!VB`r`n^*m&Q1Ww z8*^O3H5X4BSj$l!s;Jc`4(S@51s1tmXB)TWQufkgZ0vnS%ZoY9iH?O>%W&AIld315 z-vi3Y<|S)$sB@C;;RNHZmsuL?d57g2t?GAynwUt!CNl$r1F^=lcV%`SwTwoobL0o+KN%MZK z&e8qQ&bKxkL)5%seaMpBmNdPlM`44K_nY9FoLgCyr?rRWAsGwMu&+&eVrnDuhXdlE z*4_9}o3xO=gS3O+fC}n_{-8qhuPUmcmiaMku5}VE#U8sXXCPdg2G}m~N$u9=>Fw@H zd>&+hcbrPD(Ztqd#54bn8E-Eg#!hw-=Sn@&3}`JhnIyI63WKGKm~{y*1||d}(;Z`H zhfPjdpN3lvd6pw1B9Fe9C3%Z($ULXHjFO*+>Su*4wt=dgSxoTJI7|+D zAS2Of#hIailg+HOeu=f4?YU-CSOgp?aVj1Ue6D%cdW8Cbxk{r=sv1iQkIhu~z1FSA zzPavs<2Fh?n+F)#u`|<|#mWv%WM919JoY$2&*=`<@LA^jWqP4`fl=PJiLG<_Z5bd^ z)WD<=hM&}8kD_~Q;Vux}P-ZZ1f5P&e^nL|Lp`}Lc;$?@_FhCXMp-3% zKa~ybw0&OGN{l6P+5~-_lky8VOGz%8dV#N2@YZaRk*P%@00X!;+@ZDOkCrKwHVZttm9JM{@dte0}Zi=Y1(8>DoX|Fiw!^o`t;9N8F-R z>;nH&z?*LP?XaG2+QM-LUy+LyFHCC2XN13xT!M}$E@Mu|C6=s+X!+EEBREOSyq&c% zkgq|Pmg+-m1=lz^qYq7nLzBO9zlrVyb(Zql_Y)X+t=&FGq}#`J2M?6nbx-u*NNXgg zBetVbOy!t6S}{AwK6NwToC{Ll%s=q=8i+&bhXBI2zU6BE$j_qtgI(LL_r#f)&pf7V zG32Z2K6(>$g+-&SOe3l0s|BYc_FUV=&YT*K%aYa%H5Xm4mPglh&ZNs-MLkCR?$k@) zKU`S{38Y~B{GBdl^b91=*1Kh9VRr&ciMHW%?B!lzHs6_(j@wQfSbt(u&*^KoO&|Zh zD#3TPg{XILUTG=m`%=PkR-MPyWWV-<6gz~egfbgGx16WAS9AV!+sy+(XP0!k@;v{T zCjVh~l*$2j*~SW>o#DMM_pNcHaCM;Acu<)_)7S@9Z3Lb<^WqJGnu@{24^ng|zVv)4FBxF8z@z7sP7vb&v2-IK;Bl2eVjd za3?E*CD`F=NW635OK4+R#WfT5hs8hymJV~&x+&~?xGVdl_&NX9i$notT1t-InsP5e zMz_vVw|7$pOp3W7+Vhn>!I^Q7lO(Q*vDiE}>QBk#&1VrZH`HTMmq&zW#M@6cJWJZL zvz*+nRG5KZUQ{-Q9}w zO(s=mLJGt&Ru0!bVk_Hi`xh(Xi$ye^F}A=qFh2kPfrG-|xYQjuET7Jx;(NU-$Ib?fJIG2!&bJmt0EcsLYomw6j~YKaR<*6^@2h z?_Ll79eEcVl5M(8lT;BhI+3SLk6y)eMoQC$+ZJWVGR5T2N+)TLfnFLdJ#0=n_0+w| zthd3?h_3(LnXZY$7mp&?lJ^@MDn!oHZZ+(aBukxHF_SkN8>@#BCll3% zE1j}q&5eNy@tu7GRiYe%MxskvuTXc9rxh$_@=M?xXDyz(=K0sSqa1}Lvtu= z>YH3L1rFXqOWvMf^=B12wUP3Xsd(9&+fqF@25?Q2#y^9nywp;0Z!`gJ8)_mu* zg>30!$+HsI83W;BzKK(An;?Dvm_7@tKEsiZCG=>1uFbc0>wN_UyWa9U-DL_)elyQL zn0dr}R{UnZ;=JIWrjPefg1u0tYAwmjn7*&xRTWYf)EhZy_w)*Jn`h=drvPwiYGCFY z5}pgs=sV9=0bv5W*@au;na$_gtn$O2?P*J>^Mgv%GcR&a4D5OakD+k2J&(Y&%wLzw zf!QQo&w+EhwS#-QqWoO0J}I2*wF{7elWu;@+B!=R;;aq;rcvI+%9>L^x5nMA6y;ZN zamLYkda768Dlhy>kVICNM!-QzlC6Xk)LPP4SP32n@RevXZkO-cTaUHj`ThvmhVC-SL4}w9!P2p*On{XN6|H5?GOPpzL^E8o2?x@ z#hHSFqFO!9)ST&Oan?}FhG~ugk{U*ck=W=cK&tG($#4nGw|6qkqQDK9Pc-uusKJgU zj91NXow`VmLa2Xj7ZYp~Ey);BO28dWRUBop-jTHSGKMKy@xBS%3R1h^+Hn6M5QWOO zrAahSHp$Uz6wmQDTBRM8WbW-MTzzO^Tf?dtUVX!=uQa2PeCvHUwodDFKfyPnDx_i4 zL&Lx+2^7&RlzC6{Y(^|Xk3-Q{_xyv)?TV+uB%5hJzym_>pVASpo8CVgbD2AIWw_Az zk=Q$cXLf9=A~(P+lv|q^{>>{o~U_yL!We%TWL&4*Tu6qaJIcn8KQMYef-;_MgIqt^&M;i4N5>B-xl^B~tag_aZ(az1VTvs-uWgn7{Za!a>|Mm`<( zMIu6Ozz(_(?yjJ5dj9q6Ry_X+Zbj1lj7Xc$D4kdRX&f?#T=5hok1E!p;!nht5qFRB zRM&=P_+-Mp?0GC_n6Iz7l5BMNWyFy-JC&`O6>JS9N_O26b_d&`#OXGab6-Dd5KCcV zMrcqauHD*eOS}_+USGz`#ZRjRJ}$_AAz+uI*TLZe7@p_3c>^GPC)?sf^dUQb%7`gL zlsMK5!qGMlzD!AbfwD78tfIbbS^jEn#Vb6g4gvFTmF*do9uJY(TP=9_9kIJlbM=WmE7Pc2$lEt4*F{id2+t$S-i7TVASP{A(uk3rDR4nPOP|UE)Eht*cBE_<rr4?(wa~m+!hZ1)e1k_hzcO3=DS}=`P(^3GIF!XGs~r2ug}~OKZ~vx@}~m zbmaVb6hCXycw{*Dd5S6A;x_%*nHxy&tXrF`-o?kx^)Y(Kca#1^YKpSmMTRi^WvWYu zZn&z=w*e{q!}G*}VfE$d)(cmeI_`?-7kDLsrfgG-jf!B|tN; zM+q{G;Z12?v?k!~YMv&T-d|!MmRBu_u`ca|FEsP7ZPA+qQ>?U|IN4HVHFfJ#4zr!D zOEq9}&QT~p)7%P?eUugZiP|Mq#3PXgOi3dPeVQp<*#$~XUr#5TeqNH$2#mtuBPVOV z%^Tyw2BC34?TJ!c>9ZVUcIGd$muu+8B2%)uxGF%rtRXscAhh`u7~ z8L95Sar&};Q`;??xp_n-=NWhVa0Htgr0J}W^~K}H#eRxv8`m>MEg7e*4TL(|Zdpom z<+!x7MdZXvzldIY{vkH_{KMd)iVzeX*G&N)by<%`GCY?i>@-80>6!|g-p#eg@I5ap zP*T9k$_KqQs@H&#!(uumx8QMs#!p)dT^Ge#ReakyyI6lQh41U@&qUQ*XX#a1`=0tF z^sk2B9u}n>HwqyDD$X4|On&8Ja`6eWQfN_f|?OFb4 ze8@|>SKHO{>crQ)lIo<==p>)O74l+A4I$Lxy{^G2KLVXqUi%wPy}prNM!RoRk3Y$E z>o?U)7~ZKAK8~b!CFj~_8$*_x_7A?uSJyt))X%)WAO89S{L7a+bpY-MJ`4&hO^>+s zvt&pc+fK{O`?cRMq#ml^FhRM>LiN|wT!?QN<$9CVCc+fIJ2BOWi3N^pKiCrgy_os2 lD&utBr;4Hap`4&^Hy~Wr_T)#(x#PbKbv2DNO4S@f{|9phFVFx0 literal 0 HcmV?d00001 diff --git a/storm-core/src/ui/public/index.html b/storm-core/src/ui/public/index.html index 90f9edb0b75..69081d4f35f 100644 --- a/storm-core/src/ui/public/index.html +++ b/storm-core/src/ui/public/index.html @@ -40,7 +40,12 @@ -
+
+
+ + Storm Flux YAML Viewer + +
diff --git a/storm-core/src/ui/public/js/cytoscape-dagre.js b/storm-core/src/ui/public/js/cytoscape-dagre.js new file mode 100644 index 00000000000..c93288b64da --- /dev/null +++ b/storm-core/src/ui/public/js/cytoscape-dagre.js @@ -0,0 +1,192 @@ +;(function(){ 'use strict'; + + // registers the extension on a cytoscape lib ref + var register = function( cytoscape, dagre ){ + if( !cytoscape || !dagre ){ return; } // can't register if cytoscape unspecified + + var isFunction = function(o){ return typeof o === 'function'; }; + + // default layout options + var defaults = { + // dagre algo options, uses default value on undefined + nodeSep: undefined, // the separation between adjacent nodes in the same rank + edgeSep: undefined, // the separation between adjacent edges in the same rank + rankSep: undefined, // the separation between adjacent nodes in the same rank + rankDir: undefined, // 'TB' for top to bottom flow, 'LR' for left to right + minLen: function( edge ){ return 1; }, // number of ranks to keep between the source and target of the edge + edgeWeight: function( edge ){ return 1; }, // higher weight edges are generally made shorter and straighter than lower weight edges + + // general layout options + fit: true, // whether to fit to viewport + padding: 30, // fit padding + animate: false, // whether to transition the node positions + animationDuration: 500, // duration of animation in ms if enabled + animationEasing: undefined, // easing of animation if enabled + boundingBox: undefined, // constrain layout bounds; { x1, y1, x2, y2 } or { x1, y1, w, h } + ready: function(){}, // on layoutready + stop: function(){} // on layoutstop + }; + + // constructor + // options : object containing layout options + function DagreLayout( options ){ + var opts = this.options = {}; + for( var i in defaults ){ opts[i] = defaults[i]; } + for( var i in options ){ opts[i] = options[i]; } + } + + // runs the layout + DagreLayout.prototype.run = function(){ + var options = this.options; + var layout = this; + + var cy = options.cy; // cy is automatically populated for us in the constructor + var eles = options.eles; + + var getVal = function( ele, val ){ + return isFunction(val) ? val.apply( ele, [ ele ] ) : val; + }; + + var bb = options.boundingBox || { x1: 0, y1: 0, w: cy.width(), h: cy.height() }; + if( bb.x2 === undefined ){ bb.x2 = bb.x1 + bb.w; } + if( bb.w === undefined ){ bb.w = bb.x2 - bb.x1; } + if( bb.y2 === undefined ){ bb.y2 = bb.y1 + bb.h; } + if( bb.h === undefined ){ bb.h = bb.y2 - bb.y1; } + + var g = new dagre.graphlib.Graph({ + multigraph: true, + compound: true + }); + + var gObj = {}; + var setGObj = function( name, val ){ + if( val != null ){ + gObj[ name ] = val; + } + }; + + setGObj( 'nodesep', options.nodeSep ); + setGObj( 'edgesep', options.edgeSep ); + setGObj( 'ranksep', options.rankSep ); + setGObj( 'rankdir', options.rankDir ); + + g.setGraph( gObj ); + + g.setDefaultEdgeLabel(function() { return {}; }); + g.setDefaultNodeLabel(function() { return {}; }); + + // add nodes to dagre + var nodes = eles.nodes(); + for( var i = 0; i < nodes.length; i++ ){ + var node = nodes[i]; + var nbb = node.boundingBox(); + + g.setNode( node.id(), { + width: nbb.w, + height: nbb.h, + name: node.id() + } ); + + // console.log( g.node(node.id()) ); + } + + // set compound parents + for( var i = 0; i < nodes.length; i++ ){ + var node = nodes[i]; + + if( node.isChild() ){ + g.setParent( node.id(), node.parent().id() ); + } + } + + // add edges to dagre + var edges = eles.edges().stdFilter(function( edge ){ + return !edge.source().isParent() && !edge.target().isParent(); // dagre can't handle edges on compound nodes + }); + for( var i = 0; i < edges.length; i++ ){ + var edge = edges[i]; + + g.setEdge( edge.source().id(), edge.target().id(), { + minlen: getVal( edge, options.minLen ), + weight: getVal( edge, options.edgeWeight ), + name: edge.id() + }, edge.id() ); + + // console.log( g.edge(edge.source().id(), edge.target().id(), edge.id()) ); + } + + dagre.layout( g ); + + var gNodeIds = g.nodes(); + for( var i = 0; i < gNodeIds.length; i++ ){ + var id = gNodeIds[i]; + var n = g.node( id ); + + cy.getElementById(id).scratch().dagre = n; + } + + var dagreBB; + + if( options.boundingBox ){ + dagreBB = { x1: Infinity, x2: -Infinity, y1: Infinity, y2: -Infinity }; + nodes.forEach(function( node ){ + var dModel = node.scratch().dagre; + + dagreBB.x1 = Math.min( dagreBB.x1, dModel.x ); + dagreBB.x2 = Math.max( dagreBB.x2, dModel.x ); + + dagreBB.y1 = Math.min( dagreBB.y1, dModel.y ); + dagreBB.y2 = Math.max( dagreBB.y2, dModel.y ); + }); + + dagreBB.w = dagreBB.x2 - dagreBB.x1; + dagreBB.h = dagreBB.y2 - dagreBB.y1; + } else { + dagreBB = bb; + } + + var constrainPos = function( p ){ + if( options.boundingBox ){ + var xPct = (p.x - dagreBB.x1) / dagreBB.w; + var yPct = (p.y - dagreBB.y1) / dagreBB.h; + + return { + x: bb.x1 + xPct * bb.w, + y: bb.y1 + yPct * bb.h + }; + } else { + return p; + } + }; + + nodes.layoutPositions(layout, options, function(){ + var dModel = this.scratch().dagre; + + return constrainPos({ + x: dModel.x, + y: dModel.y + }); + }); + + return this; // chaining + }; + + cytoscape('layout', 'dagre', DagreLayout); + + }; + + if( typeof module !== 'undefined' && module.exports ){ // expose as a commonjs module + module.exports = register; + } + + if( typeof define !== 'undefined' && define.amd ){ // expose as an amd/requirejs module + define('cytoscape-dagre', function(){ + return register; + }); + } + + if( typeof cytoscape !== 'undefined' && typeof dagre !== 'undefined' ){ // expose to global cytoscape (i.e. window.cytoscape) + register( cytoscape, dagre ); + } + +})(); diff --git a/storm-core/src/ui/public/js/cytoscape.min.js b/storm-core/src/ui/public/js/cytoscape.min.js new file mode 100644 index 00000000000..e0e77be27c0 --- /dev/null +++ b/storm-core/src/ui/public/js/cytoscape.min.js @@ -0,0 +1,63 @@ +!function(e){if("object"==typeof exports&&"undefined"!=typeof module)module.exports=e();else if("function"==typeof define&&define.amd)define([],e);else{var t;t="undefined"!=typeof window?window:"undefined"!=typeof global?global:"undefined"!=typeof self?self:this,t.cytoscape=e()}}(function(){var define,module,exports;return function e(t,r,n){function i(o,s){if(!r[o]){if(!t[o]){var l="function"==typeof require&&require;if(!s&&l)return l(o,!0);if(a)return a(o,!0);var u=new Error("Cannot find module '"+o+"'");throw u.code="MODULE_NOT_FOUND",u}var c=r[o]={exports:{}};t[o][0].call(c.exports,function(e){var r=t[o][1][e];return i(r?r:e)},c,c.exports,e,t,r,n)}return r[o].exports}for(var a="function"==typeof require&&require,o=0;oa&&(n=a,r=i)}return r}},a=this._private.cy;if(null!=e&&null!=e.root){var o=n.string(e.root)?this.filter(e.root)[0]:e.root[0];if(null!=e.goal){var s=n.string(e.goal)?this.filter(e.goal)[0]:e.goal[0];if(null!=e.heuristic&&n.fn(e.heuristic))var l=e.heuristic;else var l=function(){return 0};if(null!=e.weight&&n.fn(e.weight))var u=e.weight;else var u=function(e){return 1};if(null!=e.directed)var c=e.directed;else var c=!1;var d=[],h=[o.id()],p={},f={},v={},g={};v[o.id()]=0,g[o.id()]=l(o);for(var y=this.edges().stdFilter(function(e){return!e.isLoop()}),m=this.nodes(),b=0;h.length>0;){var x=i(h,g),w=a.getElementById(h[x]);if(b++,w.id()==s.id()){var E=r(o.id(),s.id(),p,[]);return E.reverse(),{found:!0,distance:v[w.id()],path:t.spawn(E),steps:b}}d.push(w.id()),h.splice(x,1);var _=w.connectedEdges();c&&(_=_.stdFilter(function(e){return e.data("source")===w.id()})),_=_.intersect(y);for(var P=0;P<_.length;P++){var S=_[P],k=S.connectedNodes().stdFilter(function(e){return e.id()!==w.id()}).intersect(m);if(-1==d.indexOf(k.id())){var T=v[w.id()]+u.apply(S,[S]);-1!=h.indexOf(k.id())?Th;h++)d[u[h].id()]=h;for(var p=[],f=[],v=[],h=0;c>h;h++)u[h].id()===o.id()?p[h]=0:p[h]=1/0,f[h]=void 0;for(var g=!1,h=1;c>h;h++){g=!1;for(var y=0;yh;h++)E.push(u[h].id());var _={distanceTo:function(e){if(n.string(e))var t=s.filter(e)[0].id();else var t=e.id();return p[d[t]]},pathTo:function(e){var r=function(e,t,r,n,i,a){for(;;){if(i.push(s.getElementById(n[r])),i.push(a[r]),t===r)return i;var o=e[r];if("undefined"==typeof o)return;r=o}};if(n.string(e))var i=s.filter(e)[0].id();else var i=e.id();var a=[],l=r(f,d[o.id()],d[i],E,a,v);return null!=l&&l.reverse(),t.spawn(l)},hasNegativeWeightCycle:!1};return _}}};t.exports=a},{"../../is":83,"../../util":100}],5:[function(e,t,r){"use strict";var n=e("../../is"),i=e("../../heap"),a={betweennessCentrality:function(e){e=e||{};var t,r;n.fn(e.weight)?(r=e.weight,t=!0):t=!1;for(var a=null!=e.directed?e.directed:!1,o=this._private.cy,s=this.nodes(),l={},u={},c=0,d={set:function(e,t){u[e]=t,t>c&&(c=t)},get:function(e){return u[e]}},h=0;h0?S.edgesTo(P)[0]:P.edgesTo(S)[0];var k=r.apply(_,[_]);P=P.id(),x[P]>x[p]+k&&(x[P]=x[p]+k,w.nodes.indexOf(P)<0?w.push(P):w.updateItem(P),b[P]=0,m[P]=[]),x[P]==x[p]+k&&(b[P]=b[P]+b[p],m[P].push(p))}else for(var E=0;E0;)for(var P=y.pop(),E=0;Ea&&(a=u),i[o[l].id()]=u}return{closeness:function(e){if(0==a)return 0;if(n.string(e))var e=t.filter(e)[0].id();else var e=e.id();return i[e]/a}}},closenessCentrality:function(e){if(e=e||{},null!=e.root){if(n.string(e.root))var t=this.filter(e.root)[0];else var t=e.root[0];if(null!=e.weight&&n.fn(e.weight))var r=e.weight;else var r=function(){return 1};if(null!=e.directed&&n.bool(e.directed))var i=e.directed;else var i=!1;var a=e.harmonic;void 0===a&&(a=!0);for(var o=this.dijkstra({root:t,weight:r,directed:i}),s=0,l=this.nodes(),u=0;ud;d++){var h=a[d],p=this.degreeCentrality(i.extend({},e,{root:h}));ud;d++){var h=a[d],p=this.degreeCentrality(i.extend({},e,{root:h}));vu||!i)&&(o=u,i=l)}return{edge:i,dist:o}};v.size()>0;){var b=v.pop(),x=p(b),w=b.id();if(c[w]=x,x===Math.Infinite)break;for(var E=b.neighborhood().intersect(h),g=0;g0)for(r.unshift(t);u[i.id()];){var a=u[i.id()];r.unshift(a.edge),r.unshift(a.node),i=a.node}return o.collection(r)}}}};t.exports=a},{"../../heap":81,"../../is":83}],10:[function(e,t,r){"use strict";var n=e("../../is"),i={floydWarshall:function(e){e=e||{};var t=this.cy();if(null!=e.weight&&n.fn(e.weight))var r=e.weight;else var r=function(e){return 1};if(null!=e.directed)var i=e.directed;else var i=!1;for(var a=this.edges().stdFilter(function(e){return!e.isLoop()}),o=this.nodes(),s=o.length,l={},u=0;s>u;u++)l[o[u].id()]=u;for(var c=[],u=0;s>u;u++){for(var d=new Array(s),h=0;s>h;h++)u==h?d[h]=0:d[h]=1/0;c.push(d)}var p=[],f=[],v=function(e){for(var t=0;s>t;t++){for(var r=new Array(s),n=0;s>n;n++)r[n]=void 0;e.push(r)}};v(p),v(f);for(var u=0;um&&(c[g][y]=m,p[g][y]=y,f[g][y]=a[u])}if(!i)for(var u=0;um&&(c[g][y]=m,p[g][y]=y,f[g][y]=a[u])}for(var b=0;s>b;b++)for(var u=0;s>u;u++)for(var h=0;s>h;h++)c[u][b]+c[b][h]u;u++)x.push(o[u].id());var w={distance:function(e,r){if(n.string(e))var i=t.filter(e)[0].id();else var i=e.id();if(n.string(r))var a=t.filter(r)[0].id();else var a=r.id();return c[l[i]][l[a]]},path:function(e,r){var i=function(e,r,n,i,a){if(e===r)return t.getElementById(i[e]);if(void 0!==n[e][r]){for(var o=[t.getElementById(i[e])],s=e;e!==r;){s=e,e=n[e][r];var l=a[s][e];o.push(l),o.push(t.getElementById(i[e]))}return o}};if(n.string(e))var a=t.filter(e)[0].id();else var a=e.id();if(n.string(r))var o=t.filter(r)[0].id();else var o=r.id();var s=i(l[a],l[o],p,x,f);return t.collection(s)}};return w}};t.exports=i},{"../../is":83}],11:[function(e,t,r){"use strict";var n=e("../../util"),i={};[e("./bfs-dfs"),e("./dijkstra"),e("./kruskal"),e("./a-star"),e("./floyd-warshall"),e("./bellman-ford"),e("./kerger-stein"),e("./page-rank"),e("./degree-centrality"),e("./closeness-centrality"),e("./betweenness-centrality")].forEach(function(e){n.extend(i,e)}),t.exports=i},{"../../util":100,"./a-star":3,"./bellman-ford":4,"./betweenness-centrality":5,"./bfs-dfs":6,"./closeness-centrality":7,"./degree-centrality":8,"./dijkstra":9,"./floyd-warshall":10,"./kerger-stein":12,"./kruskal":13,"./page-rank":14}],12:[function(e,t,r){"use strict";var n=e("../../util"),i={kargerStein:function(e){var t=this;e=e||{};var r=function(e,t,r){for(var n=r[e],i=n[1],a=n[2],o=t[i],s=t[a],l=r.filter(function(e){return t[e[1]]===o&&t[e[2]]===s?!1:t[e[1]]!==s||t[e[2]]!==o}),u=0;u=n)return t;var o=Math.floor(Math.random()*t.length),s=r(o,e,t);return i(e,s,n-1,a)},a=this._private.cy,o=this.edges().stdFilter(function(e){return!e.isLoop()}),s=this.nodes(),l=s.length,u=o.length,c=Math.ceil(Math.pow(Math.log(l)/Math.LN2,2)),d=Math.floor(l/Math.sqrt(2));if(2>l)return void n.error("At least 2 nodes are required for Karger-Stein algorithm");for(var h={},p=0;l>p;p++)h[s[p].id()]=p;for(var f=[],p=0;u>p;p++){var v=o[p];f.push([p,h[v.source().id()],h[v.target().id()]])}for(var g,y=1/0,m=[],p=0;l>p;p++)m.push(p);for(var b=0;c>=b;b++){var x=m.slice(0),w=i(x,f,l,d),E=x.slice(0),_=i(x,w,d,2),P=i(E,w,d,2);_.length<=P.length&&_.lengthn;n++)r+=e[n];for(var n=0;t>n;n++)e[n]=e[n]/r};if(null!=e&&null!=e.dampingFactor)var r=e.dampingFactor;else var r=.8;if(null!=e&&null!=e.precision)var i=e.precision;else var i=1e-6;if(null!=e&&null!=e.iterations)var a=e.iterations;else var a=200;if(null!=e&&null!=e.weight&&n.fn(e.weight))var o=e.weight;else var o=function(e){return 1};for(var s=this._private.cy,l=this.edges().stdFilter(function(e){return!e.isLoop()}),u=this.nodes(),c=u.length,d=l.length,h={},p=0;c>p;p++)h[u[p].id()]=p;for(var f=[],v=[],g=(1-r)/c,p=0;c>p;p++){for(var y=[],m=0;c>m;m++)y.push(0);f.push(y),v.push(0)}for(var p=0;d>p;p++){var b=l[p],x=h[b.source().id()],w=h[b.target().id()],E=o.apply(b,[b]);f[w][x]+=E,v[x]+=E}for(var _=1/c+g,m=0;c>m;m++)if(0===v[m])for(var p=0;c>p;p++)f[p][m]=_;else for(var p=0;c>p;p++)f[p][m]=f[p][m]/v[m]+g;for(var P,S=[],k=[],p=0;c>p;p++)S.push(1),k.push(0);for(var T=0;a>T;T++){for(var D=k.slice(0),p=0;c>p;p++)for(var m=0;c>m;m++)D[p]+=f[p][m]*S[m];t(D),P=S,S=D;for(var C=0,p=0;c>p;p++)C+=Math.pow(P[p]-S[p],2);if(i>C)break}var M={rank:function(e){if(n.string(e))var t=s.filter(e)[0].id();else var t=e.id();return S[h[t]]}};return M}};t.exports=i},{"../../is":83}],15:[function(e,t,r){"use strict";var n=e("../define"),i={animate:n.animate(),animation:n.animation(),animated:n.animated(),clearQueue:n.clearQueue(),delay:n.delay(),delayAnimation:n.delayAnimation(),stop:n.stop()};t.exports=i},{"../define":44}],16:[function(e,t,r){"use strict";var n=e("../util"),i={classes:function(e){e=(e||"").match(/\S+/g)||[];for(var t=this,r=[],i={},a=0;a0&&this.spawn(r).updateStyle().trigger("class"),t},addClass:function(e){return this.toggleClass(e,!0)},hasClass:function(e){var t=this[0];return!(null==t||!t._private.classes[e])},toggleClass:function(e,t){for(var r=e.match(/\S+/g)||[],n=this,i=[],a=0,o=n.length;o>a;a++)for(var s=n[a],l=!1,u=0;u0&&this.spawn(i).updateStyle().trigger("class"),n},removeClass:function(e){return this.toggleClass(e,!1)},flashClass:function(e,t){var r=this;if(null==t)t=250;else if(0===t)return r;return r.addClass(e),setTimeout(function(){r.removeClass(e)},t),r}};t.exports=i},{"../util":100}],17:[function(e,t,r){"use strict";var n={allAre:function(e){return this.filter(e).length===this.length},is:function(e){return this.filter(e).length>0},some:function(e,t){for(var r=0;r0},allAreNeighbors:function(e){return e=this.cy().collection(e),this.neighborhood().intersect(e).length===e.length}};n.allAreNeighbours=n.allAreNeighbors,t.exports=n},{}],18:[function(e,t,r){"use strict";var n={parent:function(e){for(var t=[],r=this._private.cy,n=0;n0&&t.push(a)}return this.spawn(t,{unique:!0}).filter(e)},parents:function(e){for(var t=[],r=this.parent();r.nonempty();){for(var n=0;ne}),maxDegree:i("degree",function(e,t){return e>t}),minIndegree:i("indegree",function(e,t){return t>e}),maxIndegree:i("indegree",function(e,t){return e>t}),minOutdegree:i("outdegree",function(e,t){return t>e}),maxOutdegree:i("outdegree",function(e,t){return e>t})}),a.extend(o,{totalDegree:function(e){for(var t=0,r=this.nodes(),n=0;n0?this.add(o):this;t?l.trigger("position"):l.rtrigger("position")}return this},silentPositions:function(e){return this.positions(e,!0)},renderedPosition:function(e,t){var r=this[0],n=this.cy(),i=n.zoom(),a=n.pan(),o=s.plainObject(e)?e:void 0,l=void 0!==o||void 0!==t&&s.string(e);if(r&&r.isNode()){if(!l){var u=r._private.position;return o={x:u.x*i+a.x,y:u.y*i+a.y},void 0===e?o:o[e]}for(var c=0;c0,d=c;c&&(u=u[0]);var h=d?u._private.position:{x:0,y:0};return i={x:l.x-h.x,y:l.y-h.y},void 0===e?i:i[e]}for(var p=0;p0,d=c;c&&(u=u[0]);var h=d?u._private.position:{x:0,y:0};void 0!==t?r._private.position[e]=t+h[e]:void 0!==i&&(r._private.position={x:i.x+h.x,y:i.y+h.y})}this.rtrigger("position")}else if(!a)return;return this},renderedBoundingBox:function(e){var t=this.boundingBox(e),r=this.cy(),n=r.zoom(),i=r.pan(),a=t.x1*n+i.x,o=t.x2*n+i.x,s=t.y1*n+i.y,l=t.y2*n+i.y;return{x1:a,x2:o,y1:s,y2:l,w:o-a,h:l-s}},updateCompoundBounds:function(){function e(e){if(e.isParent()){var t=e._private,n=e.children(),i="include"===e.pstyle("compound-sizing-wrt-labels").value,a=n.boundingBox({includeLabels:i,includeShadows:!1,includeOverlays:!1,useCache:!1}),o=t.position;t.autoWidth=a.w,o.x=(a.x1+a.x2)/2,t.autoHeight=a.h,o.y=(a.y1+a.y2)/2,r.push(e)}}var t=this.cy();if(!t.styleEnabled()||!t.hasCompoundNodes())return t.collection();for(var r=[],n=this;n.nonempty();){for(var i=0;ie.x2?n:e.x2,e.y1=re.y2?i:e.y2)},h=function(e,t){return d(e,t.x1,t.y1,t.x2,t.y2)},p=function(e,t,r){return l.getPrefixedProperty(e,t,r)},f=function(e,t,r,n){var i,a,o=t._private,s=o.rstyle,l=s.arrowWidth/2,u=t.pstyle(r+"-arrow-shape").value;"none"!==u&&("source"===r?(i=s.srcX,a=s.srcY):"target"===r?(i=s.tgtX,a=s.tgtY):(i=s.midX,a=s.midY),d(e,i-l,a-l,i+l,a+l))},v=function(e,t,r,n){var i;i=r?r+"-":"";var a=t._private,o=a.rstyle,s=t.pstyle(i+"label").strValue;if(s){var l,u,c,h,f=t.pstyle("text-halign"),v=t.pstyle("text-valign"),g=p(o,"labelWidth",r),y=p(o,"labelHeight",r),m=p(o,"labelX",r),b=p(o,"labelY",r),x=t.pstyle(i+"text-margin-x").pfValue,w=t.pstyle(i+"text-margin-y").pfValue,E=t.isEdge(),_=t.pstyle(i+"text-rotation"),P=t.pstyle("text-shadow-blur").pfValue/2,S=t.pstyle("text-shadow-offset-x").pfValue,k=t.pstyle("text-shadow-offset-y").pfValue,T=t.pstyle("text-shadow-opacity").value,D=t.pstyle("text-outline-width").pfValue,C=t.pstyle("text-border-width").pfValue,M=C/2,N=y,B=g,z=B/2,I=N/2;if(E)l=m-z,u=m+z,c=b-I,h=b+I;else{switch(f.value){case"left":l=m-B,u=m;break;case"center":l=m-z,u=m+z;break;case"right":l=m,u=m+B}switch(v.value){case"top":c=b-N,h=b;break;case"center":c=b-I,h=b+I;break;case"bottom":c=b,h=b+N}}var L=E&&"autorotate"===_.strValue,O=null!=_.pfValue&&0!==_.pfValue;if(L||O){var A=L?p(a.rstyle,"labelAngle",r):_.pfValue,R=Math.cos(A),q=Math.sin(A),V=function(e,t){return e-=m,t-=b,{x:e*R-t*q+m,y:e*q+t*R+b}},F=V(l,c),j=V(l,h),X=V(u,c),Y=V(u,h);l=Math.min(F.x,j.x,X.x,Y.x),u=Math.max(F.x,j.x,X.x,Y.x),c=Math.min(F.y,j.y,X.y,Y.y),h=Math.max(F.y,j.y,X.y,Y.y)}l+=x-Math.max(D,M),u+=x+Math.max(D,M),c+=w-Math.max(D,M),h+=w+Math.max(D,M),d(e,l,c,u,h),n.includeShadows&&T>0&&(l+=-P+S,u+=+P+S,c+=-P+k,h+=+P+k,d(e,l,c,u,h))}return e},g=function(e,t){var r,n,i,a,o,s,l=e._private.cy,h=l._private,p=h.styleEnabled,g={x1:1/0,y1:1/0,x2:-(1/0),y2:-(1/0)},y=e._private,m=p?e.pstyle("display").value:"element",b=e.isNode(),x=e.isEdge(),w="none"!==m;if(w){var E=0,_=0;p&&t.includeOverlays&&(E=e.pstyle("overlay-opacity").value,0!==E&&(_=e.pstyle("overlay-padding").value));var P=0,S=0;if(p&&(P=e.pstyle("width").pfValue,S=P/2),b&&t.includeNodes){var k=y.position;o=k.x,s=k.y;var P=e.outerWidth(),T=P/2,D=e.outerHeight(),C=D/2;r=o-T-_,n=o+T+_,i=s-C-_,a=s+C+_,d(g,r,i,n,a)}else if(x&&t.includeEdges){var M=y.rstyle||{};if(p&&(r=Math.min(M.srcX,M.midX,M.tgtX),n=Math.max(M.srcX,M.midX,M.tgtX),i=Math.min(M.srcY,M.midY,M.tgtY),a=Math.max(M.srcY,M.midY,M.tgtY),r-=S,n+=S,i-=S,a+=S,d(g,r,i,n,a)),p&&"haystack"===e.pstyle("curve-style").strValue){var N=M.haystackPts;if(r=N[0].x,i=N[0].y,n=N[1].x,a=N[1].y,r>n){var B=r;r=n,n=B}if(i>a){var B=i;i=a,a=B}d(g,r-S,i-S,n+S,a+S)}else{for(var z=M.bezierPts||M.linePts||[],I=0;In){var B=r;r=n,n=B}if(i>a){var B=i;i=a,a=B}r-=S,n+=S,i-=S,a+=S,d(g,r,i,n,a)}}}if(p){if(r=g.x1,n=g.x2,i=g.y1,a=g.y2,t.includeShadows&&e.pstyle("shadow-opacity").value>0){var j=e.pstyle("shadow-blur").pfValue/2,X=e.pstyle("shadow-offset-x").pfValue,Y=e.pstyle("shadow-offset-y").pfValue;d(g,r-j+X,i-j+Y,n+j+X,a+j+Y)}d(g,r-_,i-_,n+_,a+_)}p&&t.includeEdges&&x&&(f(g,e,"mid-source",t),f(g,e,"mid-target",t),f(g,e,"source",t),f(g,e,"target",t)),p&&t.includeLabels&&(v(g,e,null,t),x&&(v(g,e,"source",t),v(g,e,"target",t)))}return g.x1=c(g.x1),g.y1=c(g.y1),g.x2=c(g.x2),g.y2=c(g.y2),g.w=c(g.x2-g.x1),g.h=c(g.y2-g.y1),g.w>0&&g.h>0&&w&&u.expandBoundingBox(g,1),g},y=function(e){return e?"t":"f"},m=function(e){var t="";return t+=y(e.incudeNodes),t+=y(e.includeEdges),t+=y(e.includeLabels),t+=y(e.includeShadows),t+=y(e.includeOverlays)},b=function(e,t){var r,n=e._private,i=e.cy().headless(),a=t===x?w:m(t);return t.useCache&&!i&&n.bbCache&&n.bbCache[a]?r=n.bbCache[a]:(r=g(e,t),i||(n.bbCache=n.bbCache||{},n.bbCache[a]=r)),r},x={includeNodes:!0,includeEdges:!0,includeLabels:!0,includeShadows:!0,includeOverlays:!0,useCache:!0},w=m(x);a.recalculateRenderedStyle=function(e){var t=this.cy(),r=t.renderer(),n=t.styleEnabled();return r&&n&&r.recalculateRenderedStyle(this,e),this},a.boundingBox=function(e){if(1===this.length&&this[0]._private.bbCache&&(void 0===e||void 0===e.useCache||e.useCache===!0))return e=void 0===e?x:n(e),b(this[0],e);var t={x1:1/0,y1:1/0, +x2:-(1/0),y2:-(1/0)};e=e||l.staticEmptyObject();var r=n(e),i=this,a=i.cy(),o=a.styleEnabled();o&&this.recalculateRenderedStyle(r.useCache);for(var s=0;sd;d++){var p=c[d];p&&""!==p&&(a._private.classes[p]=!0)}(t.style||t.css)&&e.style().applyBypass(this,t.style||t.css),(void 0===r||r)&&this.restore()};t.exports=a},{"../is":83,"../util":100}],23:[function(e,t,r){"use strict";var n=e("../define"),i={on:n.on(),one:n.on({unbindSelfOnTrigger:!0}),once:n.on({unbindAllBindersOnTrigger:!0}),off:n.off(),trigger:n.trigger(),rtrigger:function(e,t){return 0!==this.length?(this.cy().notify({type:e,eles:this}),this.trigger(e,t),this):void 0}};n.eventAliasesOn(i),t.exports=i},{"../define":44}],24:[function(e,t,r){"use strict";var n=e("../is"),i=e("../selector"),a={nodes:function(e){return this.filter(function(e,t){return t.isNode()}).filter(e)},edges:function(e){return this.filter(function(e,t){return t.isEdge()}).filter(e)},filter:function(e){if(void 0===e)return this;if(n.string(e)||n.elementOrCollection(e))return i(e).filter(this);if(n.fn(e)){for(var t=[],r=0;r1&&!i){var a=this.length-1,o=this[a],s=o._private.data.id;this[a]=void 0,this[n]=o,t.indexes[s]=n}return this.length--,this},unmerge:function(e){var t=this._private.cy;if(!e)return this;if(e&&n.string(e)){var r=e;e=t.mutableElements().filter(r)}for(var i=0;in&&(n=s,r=o)}return{value:n,ele:r}},min:function(e,t){for(var r,n=1/0,i=this,a=0;as&&(n=s,r=o)}return{value:n,ele:r}}},o=a;o.u=o["|"]=o["+"]=o.union=o.or=o.add,o["\\"]=o["!"]=o["-"]=o.difference=o.relativeComplement=o.subtract=o.not,o.n=o["&"]=o["."]=o.and=o.intersection=o.intersect,o["^"]=o["(+)"]=o["(-)"]=o.symmetricDifference=o.symdiff=o.xor,o.fnFilter=o.filterFn=o.stdFilter,o.complement=o.abscomp=o.absoluteComplement,t.exports=a},{"../is":83,"../selector":87}],25:[function(e,t,r){"use strict";var n={isNode:function(){return"nodes"===this.group()},isEdge:function(){return"edges"===this.group()},isLoop:function(){return this.isEdge()&&this.source().id()===this.target().id()},isSimple:function(){return this.isEdge()&&this.source().id()!==this.target().id()},group:function(){var e=this[0];return e?e._private.group:void 0}};t.exports=n},{}],26:[function(e,t,r){"use strict";var n=e("../util"),i=e("../is"),a=e("./element"),o={generate:function(e,t,r){for(var i=null!=r?r:n.uuid();e.hasElementWithId(i);)i=n.uuid();return i}},s=function(e,t,r){if(void 0===e||!i.core(e))return void n.error("A collection must have a reference to the core");var s={},l={},u=!1;if(t){if(t.length>0&&i.plainObject(t[0])&&!i.element(t[0])){u=!0;for(var c=[],d={},h=0,p=t.length;p>h;h++){var f=t[h];null==f.data&&(f.data={});var v=f.data;if(null==v.id)v.id=o.generate(e,f);else if(e.hasElementWithId(v.id)||d[v.id])continue;var g=new a(e,f,!1);c.push(g),d[v.id]=!0}t=c}}else t=[];this.length=0;for(var h=0,p=t.length;p>h;h++){var y=t[h];if(y){var m=y._private.data.id;(!r||r.unique&&!s[m])&&(s[m]=y,l[m]=this.length,this[this.length]=y,this.length++)}}this._private={cy:e,ids:s,indexes:l},u&&this.restore()},l=a.prototype=s.prototype;l.instanceString=function(){return"collection"},l.spawn=function(e,t,r){return i.core(e)||(r=t,t=e,e=this.cy()),new s(e,t,r)},l.spawnSelf=function(){return this.spawn(this)},l.cy=function(){return this._private.cy},l.element=function(){return this[0]},l.collection=function(){return i.collection(this)?this:new s(this._private.cy,[this])},l.unique=function(){return new s(this._private.cy,this,{unique:!0})},l.hasElementWithId=function(e){return!!this._private.ids[e]},l.getElementById=function(e){var t=this._private.cy,r=this._private.ids[e];return r?r:new s(t)},l.poolIndex=function(){var e=this._private.cy,t=e._private.elements,r=this._private.data.id;return t._private.indexes[r]},l.json=function(e){var t=this.element(),r=this.cy();if(null==t&&e)return this;if(null!=t){var a=t._private;if(i.plainObject(e)){r.startBatch(),e.data&&t.data(e.data),e.position&&t.position(e.position);var o=function(r,n,i){var o=e[r];null!=o&&o!==a[r]&&(o?t[n]():t[i]())};return o("removed","remove","restore"),o("selected","select","unselect"),o("selectable","selectify","unselectify"),o("locked","lock","unlock"),o("grabbable","grabify","ungrabify"),null!=e.classes&&t.classes(e.classes),r.endBatch(),this}if(void 0===e){var s={data:n.copy(a.data),position:n.copy(a.position),group:a.group,removed:a.removed,selected:a.selected,selectable:a.selectable,locked:a.locked,grabbable:a.grabbable,classes:null};return s.classes=Object.keys(a.classes).filter(function(e){return a.classes[e]}).join(" "),s}}},l.jsons=function(){for(var e=[],t=0;td;d++){var p=t[d];p.removed()&&(p.isNode()?u.push(p):c.push(p))}l=u.concat(c);var d,f=function(){l.splice(d,1),d--};for(d=0;dP;P++){var S=w[P],k=g[S];i.number(k)&&(k=g[S]=""+g[S]),null==k||""===k?(n.error("Can not create edge `"+y+"` with unspecified "+S),_=!0):r.hasElementWithId(k)||(n.error("Can not create edge `"+y+"` with nonexistant "+S+" `"+k+"`"),_=!0)}if(_){f();continue}var T=r.getElementById(g.source),D=r.getElementById(g.target);T._private.edges.push(x),D._private.edges.push(x),x._private.source=T,x._private.target=D}v.ids={},v.ids[y]=p,v.indexes={},v.indexes[y]=p,v.removed=!1,r.addToPool(p)}for(var d=0;d0){for(var I=new s(r,l),d=0;df;f++){var g=u[f];i(g)}var y=[];y.ids={},p.removeFromPool(d);for(var f=0;f0&&(e&&this.cy().notify({type:"remove",eles:E}),E.trigger("remove"));for(var f=0;fe&&(e=n+e),0>t&&(t=n+t);for(var i=e;i>=0&&t>i&&n>i;i++)r.push(this[i]);return this.spawn(r)},size:function(){return this.length},eq:function(e){return this[e]||this.spawn()},first:function(){return this[0]||this.spawn()},last:function(){return this[this.length-1]||this.spawn()},empty:function(){return 0===this.length},nonempty:function(){return!this.empty()},sort:function(e){if(!n.fn(e))return this;var t=this.toArray().sort(e);return this.spawn(t)},sortByZIndex:function(){return this.sort(i)},zDepth:function(){var e=this[0];if(e){var t=e._private,r=t.group;if("nodes"===r){var n=t.data.parent?e.parents().size():0;return e.isParent()?n:Number.MAX_VALUE}var i=t.source,a=t.target,o=i.zDepth(),s=a.zDepth();return Math.max(o,s,0)}}};t.exports=a},{"../is":83,"./zsort":32}],28:[function(e,t,r){"use strict";var n=e("../is"),i=e("../util"),a=e("../promise"),o={layoutPositions:function(e,t,r){var i=this.nodes(),o=this.cy();if(e.trigger({type:"layoutstart",layout:e}),e.animations=[],t.animate){for(var s=0;s0?this.add(n):this;return e?i.rtrigger("style"):i.trigger("style"),this},parsedStyle:function(e){var t=this[0];if(t.cy().styleEnabled())return t?t._private.style[e]||t.cy().style().getDefaultProperty(e):void 0},renderedStyle:function(e){var t=this.cy();if(!t.styleEnabled())return this;var r=this[0];if(r){var n=r.cy().style().getRenderedStyle(r);return void 0===e?n:n[e]}},style:function(e,t){var r=this.cy();if(!r.styleEnabled())return this;var i=!1,a=r.style();if(n.plainObject(e)){var o=e;a.applyBypass(this,o,i);var s=this.updateCompoundBounds(),l=s.length>0?this.add(s):this;l.rtrigger("style")}else if(n.string(e)){if(void 0===t){var u=this[0];return u?a.getStylePropertyValue(u,e):void 0}a.applyBypass(this,e,t,i);var s=this.updateCompoundBounds(),l=s.length>0?this.add(s):this;l.rtrigger("style")}else if(void 0===e){var u=this[0];return u?a.getRawStyle(u):void 0}return this},removeStyle:function(e){var t=this.cy();if(!t.styleEnabled())return this;var r=!1,n=t.style(),i=this;if(void 0===e)for(var a=0;a0?this.add(s):this;return l.rtrigger("style"),this},show:function(){return this.css("display","element"),this},hide:function(){return this.css("display","none"),this},visible:function(){var e=this.cy();if(!e.styleEnabled())return!0;var t=this[0],r=e.hasCompoundNodes();if(t){if("visible"!==t.pstyle("visibility").value||"element"!==t.pstyle("display").value||0===t.pstyle("width").pfValue)return!1;if("nodes"===t._private.group){if(0===t.pstyle("height").pfValue)return!1;if(!r)return!0;var n=t._private.data.parent?t.parents():null;if(n)for(var i=0;i0&&t.push(c[0]),t.push(s[0])}return this.spawn(t,{unique:!0}).filter(e)},"neighborhood"),closedNeighborhood:function(e){return this.neighborhood().add(this).filter(e)},openNeighborhood:function(e){return this.neighborhood(e)}}),l.neighbourhood=l.neighborhood,l.closedNeighbourhood=l.closedNeighborhood,l.openNeighbourhood=l.openNeighborhood,o.extend(l,{source:u(function(e){var t,r=this[0];return r&&(t=r._private.source||r.cy().collection()),t&&e?t.filter(e):t},"source"),target:u(function(e){var t,r=this[0];return r&&(t=r._private.target||r.cy().collection()),t&&e?t.filter(e):t},"target"),sources:n({attr:"source"}),targets:n({attr:"target"})}),o.extend(l,{edgesWith:u(i(),"edgesWith",!0),edgesTo:u(i({thisIsSrc:!0}),"edgesTo",!0)}),o.extend(l,{connectedEdges:u(function(e){for(var t=[],r=this,n=0;n0);return i.map(function(e){var t=e.connectedEdges().stdFilter(function(t){return e.anySame(t.source())&&e.anySame(t.target())});return e.union(t)})}}),t.exports=l},{"../is":83,"../util":100}],32:[function(e,t,r){"use strict";var n=function(e,t){var r=e.cy(),n=e.pstyle("z-index").value-t.pstyle("z-index").value,i=0,a=0,o=r.hasCompoundNodes(),s=e.isNode(),l=!s,u=t.isNode(),c=!u;o&&(i=e.zDepth(),a=t.zDepth());var d=i-a,h=0===d;return h?s&&c?1:l&&u?-1:0===n?e.poolIndex()-t.poolIndex():n:d};t.exports=n},{}],33:[function(e,t,r){"use strict";var n=e("../is"),i=e("../util"),a=e("../collection"),o=e("../collection/element"),s={add:function(e){var t,r=this;if(n.elementOrCollection(e)){var s=e;if(s._private.cy===r)t=s.restore();else{for(var l=[],u=0;uu;u++){var f=h[u],v=d[f];if(n.array(v))for(var g=0,y=v.length;y>g;g++){var m=i.extend({group:f},v[g]);l.push(m)}}t=new a(r,l)}else{var m=e;t=new o(r,m).collection()}return t},remove:function(e){if(n.elementOrCollection(e));else if(n.string(e)){var t=e;e=this.$(t)}return e.remove()},load:function(e,t,r){var a=this;a.notifications(!1);var o=a.mutableElements();o.length>0&&o.remove(),null!=e&&(n.plainObject(e)||n.array(e))&&a.add(e),a.one("layoutready",function(e){a.notifications(!0),a.trigger(e),a.notify({type:"load",eles:a.mutableElements()}),a.one("load",t),a.trigger("load")}).one("layoutstop",function(){a.one("done",r),a.trigger("done")});var s=i.extend({},a._private.options.layout);return s.eles=a.elements(),a.layout(s),this}};t.exports=s},{"../collection":26,"../collection/element":22,"../is":83,"../util":100}],34:[function(e,t,r){"use strict";var n=e("../define"),i=e("../util"),a=e("../is"),o={animate:n.animate(),animation:n.animation(),animated:n.animated(),clearQueue:n.clearQueue(),delay:n.delay(),delayAnimation:n.delayAnimation(),stop:n.stop(),addToAnimationPool:function(e){var t=this;t.styleEnabled()&&t._private.aniEles.merge(e)},stopAnimationLoop:function(){this._private.animationsRunning=!1},startAnimationLoop:function(){function e(){c._private.animationsRunning&&i.requestAnimationFrame(function(r){t(r),e()})}function t(e){function t(t,i){var s=t._private,l=s.animation.current,u=s.animation.queue,c=!1;if(0===l.length){var d=u.shift();d&&l.push(d)}for(var h=function(e){for(var t=e.length-1;t>=0;t--){var r=e[t];r()}e.splice(0,e.length)},p=l.length-1;p>=0;p--){var f=l[p],v=f._private;v.stopped?(l.splice(p,1),v.hooked=!1,v.playing=!1,v.started=!1,h(v.frames)):(v.playing||v.applying)&&(v.playing&&v.applying&&(v.applying=!1),v.started||r(t,f,e),n(t,f,e,i),a.fn(v.step)&&v.step.call(t,e),v.applying&&(v.applying=!1),h(v.frames),f.completed()&&(l.splice(p,1),v.hooked=!1,v.playing=!1,v.started=!1,h(v.completes)),c=!0)}return i||0!==l.length||0!==u.length||o.push(t),c}for(var i=c._private.aniEles,o=[],s=!1,l=0;l0){var p=i.updateCompoundBounds().spawnSelf().merge(i);c.notify({type:"draw",eles:p})}else c.notify({type:"draw"});i.unmerge(o),c.trigger("step")}function r(e,t,r){var n=a.core(e),i=!n,o=e,s=c._private.style,l=t._private;if(i){var u=o._private.position;l.startPosition=l.startPosition||{x:u.x,y:u.y},l.startStyle=l.startStyle||s.getAnimationStartStyle(o,l.style)}if(n){var d=c._private.pan;l.startPan=l.startPan||{x:d.x,y:d.y},l.startZoom=null!=l.startZoom?l.startZoom:c._private.zoom}l.started=!0,l.startTime=r-l.progress*l.duration}function n(e,t,r,n){var i=c._private.style,s=!n,l=e._private,d=t._private,h=d.easing,f=d.startTime;if(!d.easingImpl)if(null==h)d.easingImpl=p.linear;else{var v;if(a.string(h)){var g=i.parse("transition-timing-function",h);v=g.value}else v=h;var y,m;a.string(v)?(y=v,m=[]):(y=v[1],m=v.slice(2).map(function(e){return+e})),m.length>0?("spring"===y&&m.push(d.duration),d.easingImpl=p[y].apply(null,m)):d.easingImpl=p[y]}var b,x=d.easingImpl;if(b=0===d.duration?1:(r-f)/d.duration,d.applying&&(b=d.progress),0>b?b=0:b>1&&(b=1),null==d.delay){var w=d.startPosition,E=d.position,_=l.position;E&&s&&(o(w.x,E.x)&&(_.x=u(w.x,E.x,b,x)),o(w.y,E.y)&&(_.y=u(w.y,E.y,b,x)),e.trigger("position"));var P=d.startPan,S=d.pan,k=l.pan,T=null!=S&&n;T&&(o(P.x,S.x)&&(k.x=u(P.x,S.x,b,x)),o(P.y,S.y)&&(k.y=u(P.y,S.y,b,x)),e.trigger("pan"));var D=d.startZoom,C=d.zoom,M=null!=C&&n;M&&(o(D,C)&&(l.zoom=u(D,C,b,x)),e.trigger("zoom")),(T||M)&&e.trigger("viewport");var N=d.style;if(N&&N.length>0&&s){for(var B=0;Br?r=0:r>1&&(r=1);var i,o;if(i=null!=e.pfValue||null!=e.value?null!=e.pfValue?e.pfValue:e.value:e,o=null!=t.pfValue||null!=t.value?null!=t.pfValue?t.pfValue:t.value:t,a.number(i)&&a.number(o))return n(i,o,r);if(a.array(i)&&a.array(o)){for(var s=[],l=0;ld&&Math.abs(s.v)>d))break;return a?function(e){return u[e*(u.length-1)|0]}:c}}(),p={linear:function(e,t,r){return e+(t-e)*r},ease:l(.25,.1,.25,1),"ease-in":l(.42,0,1,1),"ease-out":l(0,0,.58,1),"ease-in-out":l(.42,0,.58,1),"ease-in-sine":l(.47,0,.745,.715),"ease-out-sine":l(.39,.575,.565,1),"ease-in-out-sine":l(.445,.05,.55,.95),"ease-in-quad":l(.55,.085,.68,.53),"ease-out-quad":l(.25,.46,.45,.94),"ease-in-out-quad":l(.455,.03,.515,.955),"ease-in-cubic":l(.55,.055,.675,.19),"ease-out-cubic":l(.215,.61,.355,1),"ease-in-out-cubic":l(.645,.045,.355,1),"ease-in-quart":l(.895,.03,.685,.22),"ease-out-quart":l(.165,.84,.44,1),"ease-in-out-quart":l(.77,0,.175,1),"ease-in-quint":l(.755,.05,.855,.06),"ease-out-quint":l(.23,1,.32,1),"ease-in-out-quint":l(.86,0,.07,1),"ease-in-expo":l(.95,.05,.795,.035),"ease-out-expo":l(.19,1,.22,1),"ease-in-out-expo":l(1,0,0,1),"ease-in-circ":l(.6,.04,.98,.335),"ease-out-circ":l(.075,.82,.165,1),"ease-in-out-circ":l(.785,.135,.15,.86),spring:function(e,t,r){if(0===r)return p.linear;var n=h(e,t,r);return function(e,t,r){return e+(t-e)*n(r)}},"cubic-bezier":function(e,t,r,n){return l(e,t,r,n)}}}}};t.exports=o},{"../define":44,"../is":83,"../util":100}],35:[function(e,t,r){"use strict";var n=e("../define"),i={on:n.on(),one:n.on({unbindSelfOnTrigger:!0}),once:n.on({unbindAllBindersOnTrigger:!0}),off:n.off(),trigger:n.trigger()};n.eventAliasesOn(i),t.exports=i},{"../define":44}],36:[function(e,t,r){"use strict";var n={png:function(e){var t=this._private.renderer;return e=e||{},t.png(e)},jpg:function(e){var t=this._private.renderer;return e=e||{},e.bg=e.bg||"#fff",t.jpg(e)}};n.jpeg=n.jpg,t.exports=n},{}],37:[function(e,t,r){"use strict";var n=e("../window"),i=e("../util"),a=e("../collection"),o=e("../is"),s=e("../promise"),l=e("../define"),u=function(e){var t=this;e=i.extend({},e);var r=e.container;r&&!o.htmlElement(r)&&o.htmlElement(r[0])&&(r=r[0]);var l=r?r._cyreg:null;l=l||{},l&&l.cy&&(l.cy.destroy(),l={});var u=l.readies=l.readies||[];r&&(r._cyreg=l),l.cy=t;var c=void 0!==n&&void 0!==r&&!e.headless,d=e;d.layout=i.extend({name:c?"grid":"null"},d.layout),d.renderer=i.extend({name:c?"canvas":"null"},d.renderer);var h=function(e,t,r){return void 0!==t?t:void 0!==r?r:e},p=this._private={container:r,ready:!1,initrender:!1,options:d,elements:new a(this),listeners:[],aniEles:new a(this),scratch:{},layout:null,renderer:null,notificationsEnabled:!0,minZoom:1e-50,maxZoom:1e50,zoomingEnabled:h(!0,d.zoomingEnabled),userZoomingEnabled:h(!0,d.userZoomingEnabled),panningEnabled:h(!0,d.panningEnabled),userPanningEnabled:h(!0,d.userPanningEnabled),boxSelectionEnabled:h(!0,d.boxSelectionEnabled),autolock:h(!1,d.autolock,d.autolockNodes),autoungrabify:h(!1,d.autoungrabify,d.autoungrabifyNodes),autounselectify:h(!1,d.autounselectify),styleEnabled:void 0===d.styleEnabled?c:d.styleEnabled,zoom:o.number(d.zoom)?d.zoom:1,pan:{x:o.plainObject(d.pan)&&o.number(d.pan.x)?d.pan.x:0,y:o.plainObject(d.pan)&&o.number(d.pan.y)?d.pan.y:0},animation:{current:[],queue:[]},hasCompoundNodes:!1},f=d.selectionType;void 0===f||"additive"!==f&&"single"!==f?p.selectionType="single":p.selectionType=f,o.number(d.minZoom)&&o.number(d.maxZoom)&&d.minZoom0?d.wheelSensitivity:1,motionBlur:void 0===d.motionBlur?!1:d.motionBlur,motionBlurOpacity:void 0===d.motionBlurOpacity?.05:d.motionBlurOpacity,pixelRatio:o.number(d.pixelRatio)&&d.pixelRatio>0?d.pixelRatio:void 0,desktopTapThreshold:void 0===d.desktopTapThreshold?4:d.desktopTapThreshold,touchTapThreshold:void 0===d.touchTapThreshold?8:d.touchTapThreshold},d.renderer)),v([d.style,d.elements],function(e){var r=e[0],n=e[1];p.styleEnabled&&t.setStyle(r),d.initrender&&(t.on("initrender",d.initrender),t.on("initrender",function(){p.initrender=!0})),t.load(n,function(){t.startAnimationLoop(),p.ready=!0,o.fn(d.ready)&&t.on("ready",d.ready);for(var e=0;e0;)t.removeChild(t.childNodes[0]);e._private.renderer=null},onRender:function(e){return this.on("render",e)},offRender:function(e){return this.off("render",e)}};i.invalidateDimensions=i.resize,t.exports=i},{"../util":100}],41:[function(e,t,r){"use strict";var n=e("../is"),i=e("../collection"),a={collection:function(e,t){return n.string(e)?this.$(e):n.elementOrCollection(e)?e.collection():n.array(e)?new i(this,e,t):new i(this)},nodes:function(e){var t=this.$(function(){return this.isNode()});return e?t.filter(e):t},edges:function(e){var t=this.$(function(){return this.isEdge()});return e?t.filter(e):t},$:function(e){var t=this._private.elements;return e?t.filter(e):t.spawnSelf()},mutableElements:function(){return this._private.elements}};a.elements=a.filter=a.$,t.exports=a},{"../collection":26,"../is":83}],42:[function(e,t,r){"use strict";var n=e("../is"),i=e("../style"),a={style:function(e){if(e){var t=this.setStyle(e);t.update()}return this._private.style},setStyle:function(e){var t=this._private;return n.stylesheet(e)?t.style=e.generateStyle(this):n.array(e)?t.style=i.fromJson(this,e):n.string(e)?t.style=i.fromString(this,e):t.style=i(this),t.style}};t.exports=a},{"../is":83,"../style":92}],43:[function(e,t,r){"use strict";var n=e("../is"),i=e("../window"),a={autolock:function(e){return void 0===e?this._private.autolock:(this._private.autolock=!!e,this)},autoungrabify:function(e){return void 0===e?this._private.autoungrabify:(this._private.autoungrabify=!!e,this)},autounselectify:function(e){return void 0===e?this._private.autounselectify:(this._private.autounselectify=!!e,this)},panningEnabled:function(e){return void 0===e?this._private.panningEnabled:(this._private.panningEnabled=!!e,this)},userPanningEnabled:function(e){return void 0===e?this._private.userPanningEnabled:(this._private.userPanningEnabled=!!e,this)},zoomingEnabled:function(e){return void 0===e?this._private.zoomingEnabled:(this._private.zoomingEnabled=!!e,this)},userZoomingEnabled:function(e){return void 0===e?this._private.userZoomingEnabled:(this._private.userZoomingEnabled=!!e,this)},boxSelectionEnabled:function(e){return void 0===e?this._private.boxSelectionEnabled:(this._private.boxSelectionEnabled=!!e,this)},pan:function(){var e,t,r,i,a,o=arguments,s=this._private.pan;switch(o.length){case 0:return s;case 1:if(n.string(o[0]))return e=o[0],s[e];if(n.plainObject(o[0])){if(!this._private.panningEnabled)return this;r=o[0],i=r.x,a=r.y,n.number(i)&&(s.x=i),n.number(a)&&(s.y=a),this.trigger("pan viewport")}break;case 2:if(!this._private.panningEnabled)return this;e=o[0],t=o[1],"x"!==e&&"y"!==e||!n.number(t)||(s[e]=t),this.trigger("pan viewport")}return this.notify({type:"viewport"}),this},panBy:function(e){var t,r,i,a,o,s=arguments,l=this._private.pan;if(!this._private.panningEnabled)return this;switch(s.length){case 1:n.plainObject(s[0])&&(i=s[0],a=i.x,o=i.y,n.number(a)&&(l.x+=a),n.number(o)&&(l.y+=o),this.trigger("pan viewport"));break;case 2:t=s[0],r=s[1],"x"!==t&&"y"!==t||!n.number(r)||(l[t]+=r),this.trigger("pan viewport")}return this.notify({type:"viewport"}),this},fit:function(e,t){var r=this.getFitViewport(e,t);if(r){var n=this._private;n.zoom=r.zoom,n.pan=r.pan,this.trigger("pan zoom viewport"),this.notify({type:"viewport"})}return this},getFitViewport:function(e,t){if(n.number(e)&&void 0===t&&(t=e,e=void 0),this._private.panningEnabled&&this._private.zoomingEnabled){var r;if(n.string(e)){var i=e;e=this.$(i)}else if(n.boundingBox(e)){var a=e;r={x1:a.x1,y1:a.y1,x2:a.x2,y2:a.y2},r.w=r.x2-r.x1,r.h=r.y2-r.y1}else n.elementOrCollection(e)||(e=this.mutableElements());r=r||e.boundingBox();var o,s=this.width(),l=this.height();if(t=n.number(t)?t:0,!isNaN(s)&&!isNaN(l)&&s>0&&l>0&&!isNaN(r.w)&&!isNaN(r.h)&&r.w>0&&r.h>0){o=Math.min((s-2*t)/r.w,(l-2*t)/r.h),o=o>this._private.maxZoom?this._private.maxZoom:o,o=othis._private.maxZoom?this._private.maxZoom:r,r=rt.maxZoom||!t.zoomingEnabled?o=!0:(t.zoom=l,a.push("zoom"))}if(i&&(!o||!e.cancelOnFailedZoom)&&t.panningEnabled){var u=e.pan;n.number(u.x)&&(t.pan.x=u.x,s=!1),n.number(u.y)&&(t.pan.y=u.y,s=!1),s||a.push("pan")}return a.length>0&&(a.push("viewport"),this.trigger(a.join(" ")),this.notify({type:"viewport"})),this},center:function(e){var t=this.getCenterPan(e);return t&&(this._private.pan=t,this.trigger("pan viewport"),this.notify({type:"viewport"})),this},getCenterPan:function(e,t){if(this._private.panningEnabled){if(n.string(e)){var r=e;e=this.mutableElements().filter(r)}else n.elementOrCollection(e)||(e=this.mutableElements());var i=e.boundingBox(),a=this.width(),o=this.height();t=void 0===t?this._private.zoom:t;var s={x:(a-t*(i.x1+i.x2))/2,y:(o-t*(i.y1+i.y2))/2};return s}},reset:function(){return this._private.panningEnabled&&this._private.zoomingEnabled?(this.viewport({pan:{x:0,y:0},zoom:1}),this):this},invalidateSize:function(){this._private.sizeCache=null},size:function(){var e=this._private,t=e.container;return e.sizeCache=e.sizeCache||(t?function(){var e=t.getBoundingClientRect(),r=i.getComputedStyle(t),n=function(e){return parseFloat(r.getPropertyValue(e))};return{width:e.width-n("padding-left")-n("padding-right")-n("border-left-width")-n("border-right-width"),height:e.height-n("padding-top")-n("padding-bottom")-n("border-top-width")-n("border-bottom-width")}}():{width:1,height:1})},width:function(){return this.size().width},height:function(){return this.size().height},extent:function(){var e=this._private.pan,t=this._private.zoom,r=this.renderedExtent(),n={x1:(r.x1-e.x)/t,x2:(r.x2-e.x)/t,y1:(r.y1-e.y)/t,y2:(r.y2-e.y)/t};return n.w=n.x2-n.x1,n.h=n.y2-n.y1,n},renderedExtent:function(){var e=this.width(),t=this.height();return{x1:0,y1:0,x2:e,y2:t,w:e,h:t}}};a.centre=a.center,a.autolockNodes=a.autolock,a.autoungrabifyNodes=a.autoungrabify,t.exports=a},{"../is":83,"../window":107}],44:[function(e,t,r){"use strict";var n=e("./util"),i=e("./is"),a=e("./selector"),o=e("./promise"),s=e("./event"),l=e("./animation"),u={data:function(e){var t={field:"data",bindingEvent:"data",allowBinding:!1,allowSetting:!1,allowGetting:!1,settingEvent:"data",settingTriggersEvent:!1,triggerFnName:"trigger",immutableKeys:{},updateStyle:!1,onSet:function(e){},canSet:function(e){return!0}};return e=n.extend({},t,e),function(t,r){var n=e,a=this,o=void 0!==a.length,s=o?a:[a],l=o?a[0]:a;if(i.string(t)){if(n.allowGetting&&void 0===r){var u;return l&&(u=l._private[n.field][t]),u}if(n.allowSetting&&void 0!==r){var c=!n.immutableKeys[t];if(c){for(var d=0,h=s.length;h>d;d++)n.canSet(s[d])&&(s[d]._private[n.field][t]=r);n.updateStyle&&a.updateStyle(),n.onSet(a),n.settingTriggersEvent&&a[n.triggerFnName](n.settingEvent)}}}else if(n.allowSetting&&i.plainObject(t)){for(var p,f,v=t,g=Object.keys(v),d=0;du;u++){var c=s[u];if(!i.emptyString(c)){var d=!r.immutableKeys[c];if(d)for(var h=0,p=o.length;p>h;h++)o[h]._private[r.field][c]=void 0}}r.triggerEvent&&n[r.triggerFnName](r.event)}else if(void 0===t){for(var h=0,p=o.length;p>h;h++)for(var f=o[h]._private[r.field],s=Object.keys(f),u=0;u0:void 0}},clearQueue:function(e){var t={};return e=n.extend({},t,e),function(){var e=this,t=void 0!==e.length,r=t?e:[e],n=this._private.cy||this;if(!n.styleEnabled())return this;for(var i=0;i0;){var g=n.collection();i.bfs({roots:v[0],visit:function(e,t,r,n,i){g=g.add(r)},directed:!1}),v=v.not(g),f.push(g)}e=n.collection();for(var d=0;dD;){for(var C=k.shift(),M=C.neighborhood().nodes(),N=!1,d=0;dd;d++)for(var B=x[d],R=B.length,q=0;R>q;q++){var p=B[q],V=p._private.scratch.breadthfirst,F=I(p);F&&(V.intEle=F,A.push(p))}for(var d=0;dx.length-1;)x.push([]);x[X].push(p),V.depth=X,V.index=x[X].length-1}z()}var Y=0;if(r.avoidOverlap){for(var d=0;dc||0===t)&&(n+=u/d,i++)}return i=Math.max(1,i),n/=i,0===i&&(n=void 0),Z[e.id()]=n,n},Q=function(e,t){var r=G(e),n=G(t);return r-n},K=0;3>K;K++){for(var d=0;d0&&x[0].length<=3?c/2:0),h=2*Math.PI/x[i].length*a;return 0===i&&1===x[0].length&&(d=1),{x:ee.x+d*Math.cos(h),y:ee.y+d*Math.sin(h)}}return{x:ee.x+(a+1-(o+1)/2)*s,y:(i+1)*l}}var p={x:ee.x+(a+1-(o+1)/2)*s,y:(i+1)*l};return t?p:p},re={},d=x.length-1;d>=0;d--)for(var B=x[d],q=0;q1&&t.avoidOverlap){p*=1.75;var b=Math.cos(h)-Math.cos(0),x=Math.sin(h)-Math.sin(0),w=Math.sqrt(p*p/(b*b+x*x));l=Math.max(w,l)}var E=function(e,r){var n=t.startAngle+e*h*(i?1:-1),a=l*Math.cos(n),o=l*Math.sin(n),s={x:c.x+a,y:c.y+o};return s};return s.layoutPositions(this,t,E),this},t.exports=n},{"../../is":83,"../../math":85,"../../util":100}],50:[function(e,t,r){"use strict";function n(e){this.options=i.extend({},o,e)}var i=e("../../util"),a=e("../../math"),o={fit:!0,padding:30,startAngle:1.5*Math.PI,sweep:void 0,clockwise:!0,equidistant:!1,minNodeSpacing:10,boundingBox:void 0,avoidOverlap:!0,height:void 0,width:void 0,concentric:function(e){return e.degree()},levelWidth:function(e){return e.maxDegree()/4},animate:!1,animationDuration:500,animationEasing:void 0,ready:void 0,stop:void 0};n.prototype.run=function(){for(var e=this.options,t=e,r=void 0!==t.counterclockwise?!t.counterclockwise:t.clockwise,n=e.cy,i=t.eles,o=i.nodes().not(":parent"),s=a.makeBoundingBox(t.boundingBox?t.boundingBox:{x1:0,y1:0,w:n.width(),h:n.height()}),l={x:s.x1+s.w/2,y:s.y1+s.h/2},u=[],c=t.startAngle,d=0,h=0;h0){var x=Math.abs(m[0].value-b.value);x>=g&&(m=[],y.push(m))}m.push(b)}var w=d+t.minNodeSpacing;if(!t.avoidOverlap){var E=y.length>0&&y[0].length>1,_=Math.min(s.w,s.h)/2-w,P=_/(y.length+E?1:0);w=Math.min(w,P)}for(var S=0,h=0;h1&&t.avoidOverlap){var C=Math.cos(D)-Math.cos(0),M=Math.sin(D)-Math.sin(0),N=Math.sqrt(w*w/(C*C+M*M));S=Math.max(N,S)}k.r=S,S+=w}if(t.equidistant){for(var B=0,S=0,h=0;ha;a++)for(var o=e.layoutNodes[e.idToIndex[n[a]]],l=a+1;i>l;l++){var u=e.layoutNodes[e.idToIndex[n[l]]];s(o,u,e,t)}},s=function(e,t,r,n){var i=e.cmptId,a=t.cmptId;if(i===a||r.isCompound){var o=t.positionX-e.positionX,s=t.positionY-e.positionY;if(0!==o||0!==s){var c=l(e,t,o,s);if(c>0)var d=n.nodeOverlap*c,h=Math.sqrt(o*o+s*s),p=d*o/h,f=d*s/h;else var v=u(e,o,s),g=u(t,-1*o,-1*s),y=g.x-v.x,m=g.y-v.y,b=y*y+m*m,h=Math.sqrt(b),d=(e.nodeRepulsion+t.nodeRepulsion)/b,p=d*y/h,f=d*m/h;e.isLocked||(e.offsetX-=p,e.offsetY-=f),t.isLocked||(t.offsetX+=p,t.offsetY+=f)}}},l=function(e,t,r,n){if(r>0)var i=e.maxX-t.minX;else var i=t.maxX-e.minX;if(n>0)var a=e.maxY-t.minY;else var a=t.maxY-e.minY;return i>=0&&a>=0?Math.sqrt(i*i+a*a):0},u=function(e,t,r){var n=e.positionX,i=e.positionY,a=e.height||1,o=e.width||1,s=r/t,l=a/o,u={};do{if(0===t&&r>0){u.x=n,u.y=i+a/2;break}if(0===t&&0>r){u.x=n,u.y=i+a/2;break}if(t>0&&s>=-1*l&&l>=s){u.x=n+o/2,u.y=i+o*r/2/t;break}if(0>t&&s>=-1*l&&l>=s){u.x=n-o/2,u.y=i-o*r/2/t;break}if(r>0&&(-1*l>=s||s>=l)){u.x=n+a*t/2/r,u.y=i+a/2;break}if(0>r&&(-1*l>=s||s>=l)){u.x=n-a*t/2/r,u.y=i-a/2;break}}while(!1);return u},c=function(e,t){for(var r=0;rc;c++){var d=e.layoutNodes[e.idToIndex[i[c]]];if(!d.isLocked){var h=o-d.positionX,p=s-d.positionY,f=Math.sqrt(h*h+p*p);if(f>r){var v=t.gravity*h/f,g=t.gravity*p/f;d.offsetX+=v,d.offsetY+=g}}}}},h=function(e,t){var r=[],n=0,i=-1;for(r.push.apply(r,e.graphSet[0]),i+=e.graphSet[0].length;i>=n;){var a=r[n++],o=e.idToIndex[a],s=e.layoutNodes[o],l=s.children;if(0r)var i={x:r*e/n,y:r*t/n};else var i={x:e,y:t};return i},v=function(e,t){var r=e.parentId;if(null!=r){var n=t.layoutNodes[t.idToIndex[r]],i=!1;return(null==n.maxX||e.maxX+n.padRight>n.maxX)&&(n.maxX=e.maxX+n.padRight,i=!0),(null==n.minX||e.minX-n.padLeftn.maxY)&&(n.maxY=e.maxY+n.padBottom,i=!0),(null==n.minY||e.minY-n.padTopy&&(f+=g+t.componentSpacing,p=0,v=0,g=0)}}},y=function(e){return i?!1:(a(r,n,e),r.temperature=r.temperature*n.coolingFactor,!(r.temperature=b;){var _=m[b++],P=a.idToIndex[_],f=a.layoutNodes[P],S=f.children;if(S.length>0){a.graphSet.push(S);for(var c=0;cn.count?0:n.graph},h=function(e,t,r,n){var i=n.graphSet[r];if(-1s){var v=d(),g=h();(v-1)*g>=s?d(v-1):(g-1)*v>=s&&h(g-1)}else for(;s>c*u;){var v=d(),g=h();(g+1)*v>=s?h(g+1):d(v+1)}var y=o.w/c,m=o.h/u;if(t.condense&&(y=0,m=0),t.avoidOverlap)for(var b=0;b=c&&(M=0,C++)},B={},b=0;b=o&&s>=e&&t>=l&&u>=t;return c},o=function(e,t,r,n,i){var a=e*Math.cos(n)-t*Math.sin(n),o=e*Math.sin(n)+t*Math.cos(n),s=a*r,l=o*r,u=s+i.x,c=l+i.y;return{x:u,y:c}},s=function(e,t,r,n){for(var i=[],a=0;at))if(d){if(d.pstyle("z-index").value===e.pstyle("z-index").value)for(var r=0;r(l=i.sqdistToFiniteLine(e,t,P[S],P[S+1],P[S+2],P[S+3])))return a(n,l),!0}else if("bezier"===c.edgeType||"multibezier"===c.edgeType||"self"===c.edgeType||"compound"===c.edgeType)for(var P=c.allpts,S=0;S+5(l=i.sqdistToQuadraticBezier(e,t,P[S],P[S+1],P[S+2],P[S+3],P[S+4],P[S+5])))return a(n,l),!0;if(_())for(var x=x||o.source,w=w||o.target,k=p.getArrowWidth(d),T=[{name:"source",x:c.arrowStartX,y:c.arrowStartY,angle:c.srcArrowAngle},{name:"target",x:c.arrowEndX,y:c.arrowEndY,angle:c.tgtArrowAngle},{name:"mid-source",x:c.midX,y:c.midY,angle:c.midsrcArrowAngle},{name:"mid-target",x:c.midX,y:c.midY,angle:c.midtgtArrowAngle}],S=0;S0&&(s(x),s(w))}}function u(e,t,r){return o.getPrefixedProperty(e,t,r)}function c(r,n){var o,s=r._private,l=w;o=n?n+"-":"";var c=r.pstyle(o+"label").value,d="yes"===r.pstyle("text-events").strValue;if(d&&c){var h=s.rstyle,p=r.pstyle("text-border-width").pfValue,f=u(h,"labelWidth",n)+p/2+2*l,v=u(h,"labelHeight",n)+p/2+2*l,g=u(h,"labelX",n),y=u(h,"labelY",n),m=u(s.rscratch,"labelAngle",n),b=g-f/2,x=g+f/2,E=y-v/2,_=y+v/2;if(m){var P=Math.cos(m),S=Math.sin(m),k=function(e,t){return e-=g,t-=y,{x:e*P-t*S+g,y:e*S+t*P+y}},T=k(b,E),D=k(b,_),C=k(x,E),M=k(x,_),N=[T.x,T.y,C.x,C.y,M.x,M.y,D.x,D.y];if(i.pointInsidePolygonPoints(e,t,N))return a(r),!0}else{var B={w:f,h:v,x1:b,x2:x,y1:E,y2:_};if(i.inBoundingBox(B,e,t))return a(r),!0}}}for(var d,h,p=this,f=this,v=f.getCachedZSortedEles(),g=[],y=f.cy.zoom(),m=f.cy.hasCompoundNodes(),b=(n?24:8)/y,x=(n?8:2)/y,w=(n?8:2)/y,E=1/0,_=v.length-1;_>=0;_--){var P=v[_];P.isNode()?s(P)||c(P):l(P)||c(P)||c(P,"source")||c(P,"target")}return g},u.getAllInBox=function(e,t,r,n){var a=this.getCachedZSortedEles(),o=a.nodes,s=a.edges,l=[],u=Math.min(e,r),c=Math.max(e,r),d=Math.min(t,n),h=Math.max(t,n);e=u,r=c,t=d,n=h;for(var p=i.makeBoundingBox({x1:e,y1:t,x2:r,y2:n}),f=0;fv;v++)e(p,d[o*h+v],d[o*h+v+1],a.bezierProjPcts[v],a.bezierProjPcts[v+1]);e(p,d[o*h+h-1],p.p2,a.bezierProjPcts[h-1],1)}return u.cache=t},c=function(r){var a,o="source"===r;if(s[r]){var c=e.pstyle(r+"-text-offset").pfValue,d=function(e,t){var r=t.x-e.x,n=t.y-e.y;return Math.atan(n/r)},h=function(e,t,r,n){var a=i.bound(0,n-.001,1),o=i.bound(0,n+.001,1),s=i.qbezierPtAt(e,t,r,a),l=i.qbezierPtAt(e,t,r,o);return d(s,l)};switch(n.edgeType){case"self":case"compound":case"bezier":case"multibezier":for(var p,f=u(),v=0,g=0,y=0;y=c||w){p={cp:m,segment:x};break}}if(p)break}var m=p.cp,x=p.segment,E=(c-v)/x.length,_=x.t1-x.t0,P=o?x.t0+_*E:x.t1-_*E;P=i.bound(0,P,1),t=i.qbezierPtAt(m.p0,m.p1,m.p2,P),a=h(m.p0,m.p1,m.p2,P,t);break;case"straight":case"segments":case"haystack":for(var S,k,T,D,C=0,M=n.allpts.length,y=0;M>y+3&&(o?(T={x:n.allpts[y],y:n.allpts[y+1]},D={x:n.allpts[y+2],y:n.allpts[y+3]}):(T={x:n.allpts[M-2-y],y:n.allpts[M-1-y]},D={x:n.allpts[M-4-y],y:n.allpts[M-3-y]}),S=i.dist(T,D),k=C,C+=S,!(C>=c));y+=2);var N=c-k,P=N/S;P=i.bound(0,P,1),t=i.lineAt(T,D,P),a=d(T,D)}l("labelX",r,t.x),l("labelY",r,t.y),l("labelAutoAngle",r,a)}};c("source"),c("target"),this.applyLabelDimensions(e)}},u.applyLabelDimensions=function(e){this.applyPrefixedLabelDimensions(e),e.isEdge()&&(this.applyPrefixedLabelDimensions(e,"source"),this.applyPrefixedLabelDimensions(e,"target"))},u.applyPrefixedLabelDimensions=function(e,t){var r=e._private,n=this.getLabelText(e,t),i=this.calculateLabelDimensions(e,n);o.setPrefixedProperty(r.rstyle,"labelWidth",t,i.width),o.setPrefixedProperty(r.rscratch,"labelWidth",t,i.width),o.setPrefixedProperty(r.rstyle,"labelHeight",t,i.height),o.setPrefixedProperty(r.rscratch,"labelHeight",t,i.height)},u.getLabelText=function(e,t){var r=e._private,n=t?t+"-":"",i=e.pstyle(n+"label").strValue,a=e.pstyle("text-transform").value,s=function(e,n){return n?(o.setPrefixedProperty(r.rscratch,e,t,n),n):o.getPrefixedProperty(r.rscratch,e,t)};if("none"==a||("uppercase"==a?i=i.toUpperCase():"lowercase"==a&&(i=i.toLowerCase())),"wrap"===e.pstyle("text-wrap").value){var l=s("labelKey");if(l&&s("labelWrapKey")===l)return s("labelWrapCachedText");for(var u=i.split("\n"),c=e.pstyle("text-max-width").pfValue,d=[],h=0;hc){for(var g=p.split(/\s+/),y="",m=0;m=E?y+=b+" ":(d.push(y),y=b+" ")}y.match(/^\s+$/)||d.push(y)}else d.push(p)}s("labelWrapCachedLines",d),i=s("labelWrapCachedText",d.join("\n")),s("labelWrapKey",l)}return i},u.calculateLabelDimensions=function(e,t,r){var n=this,i=e._private.labelStyleKey+"$@$"+t;r&&(i+="$@$"+r);var a=n.labelDimCache||(n.labelDimCache={});if(a[i])return a[i];var o=1,s=e.pstyle("font-style").strValue,l=o*e.pstyle("font-size").pfValue+"px",u=e.pstyle("font-family").strValue,c=e.pstyle("font-weight").strValue,d=this.labelCalcDiv;d||(d=this.labelCalcDiv=document.createElement("div"),document.body.appendChild(d));var h=d.style;return h.fontFamily=u,h.fontStyle=s,h.fontSize=l,h.fontWeight=c,h.position="absolute",h.left="-9999px",h.top="-9999px",h.zIndex="-1",h.visibility="hidden",h.pointerEvents="none",h.padding="0",h.lineHeight="1","wrap"===e.pstyle("text-wrap").value?h.whiteSpace="pre":h.whiteSpace="normal",d.textContent=t,a[i]={width:Math.ceil(d.clientWidth/o),height:Math.ceil(d.clientHeight/o)},a[i]},u.recalculateEdgeProjections=function(e){this.findEdgeControlPoints(e)},u.findEdgeControlPoints=function(e){if(e&&0!==e.length){for(var t,r=this,n=r.cy,o=n.hasCompoundNodes(),s={},l=[],u=[],c=0;cy?y+"$-$"+g:g+"$-$"+y,v&&(t="unbundled$-$"+p.id),null==s[t]&&(s[t]=[],l.push(t)),s[t].push(d),v&&(s[t].hasUnbundled=!0)}else u.push(d)}for(var m,b,x,w,E,_,P,S,k,T,D,C,M,N,B=0;Bb.id()){var I=m;m=b,b=I}if(x=m._private,w=b._private,E=x.position,_=w.position,P=m.outerWidth(),S=m.outerHeight(),k=b.outerWidth(),T=b.outerHeight(),D=r.nodeShapes[this.getNodeShape(m)],C=r.nodeShapes[this.getNodeShape(b)],N=!1,z.length>1&&m!==b||z.hasUnbundled){var L=D.intersectLine(E.x,E.y,P,S,_.x,_.y,0),O=C.intersectLine(_.x,_.y,k,T,E.x,E.y,0),A={x1:L[0],x2:O[0],y1:L[1],y2:O[1]},R={x1:E.x,x2:_.x,y1:E.y,y2:_.y},q=O[1]-L[1],V=O[0]-L[0],F=Math.sqrt(V*V+q*q),j={x:V,y:q},X={x:j.x/F,y:j.y/F};M={x:-X.y,y:X.x},C.checkPoint(L[0],L[1],0,k,T,_.x,_.y)&&D.checkPoint(O[0],O[1],0,P,S,E.x,E.y)&&(M={},N=!0)}for(var d,Y,W,c=0;cze;ze++){var Ie=Me[ze],Le=Ne[ze],Oe=1-Ie,Ae=Ie,Re="node-position"===xe?R:A,qe={x:Re.x1*Oe+Re.x2*Ae,y:Re.y1*Oe+Re.y2*Ae};W.segpts.push(qe.x+M.x*Le,qe.y+M.y*Le)}}else if(z.length%2!==1||c!==Math.floor(z.length/2)||v){var Ve=v;W.edgeType=Ve?"multibezier":"bezier",W.ctrlpts=[];for(var Fe=0;K>Fe;Fe++){var je,Xe=(.5-z.length/2+c)*J,Ye=i.signum(Xe);Ve&&(ee=G?G.pfValue[Fe]:J,te=Q.value[Fe]),je=v?ee:void 0!==ee?Ye*ee:void 0;var We=void 0!==je?je:Xe,Oe=1-te,Ae=te,Re="node-position"===xe?R:A,qe={x:Re.x1*Oe+Re.x2*Ae,y:Re.y1*Oe+Re.y2*Ae};W.ctrlpts.push(qe.x+M.x*We,qe.y+M.y*We)}}else W.edgeType="straight";this.findEndpoints(d);var $e=!a.number(W.startX)||!a.number(W.startY),He=!a.number(W.arrowStartX)||!a.number(W.arrowStartY),Ue=!a.number(W.endX)||!a.number(W.endY),Ze=!a.number(W.arrowEndX)||!a.number(W.arrowEndY),Ge=3,Qe=this.getArrowWidth(d.pstyle("width").pfValue)*this.arrowShapeWidth,Ke=Ge*Qe;if("bezier"===W.edgeType){var Je=i.dist({x:W.ctrlpts[0],y:W.ctrlpts[1]},{x:W.startX,y:W.startY}),et=Ke>Je,tt=i.dist({x:W.ctrlpts[0],y:W.ctrlpts[1]},{x:W.endX,y:W.endY}),rt=Ke>tt,nt=!1;if($e||He||et){nt=!0;var it={x:W.ctrlpts[0]-E.x,y:W.ctrlpts[1]-E.y},at=Math.sqrt(it.x*it.x+it.y*it.y),ot={x:it.x/at,y:it.y/at},st=Math.max(P,S),lt={x:W.ctrlpts[0]+2*ot.x*st,y:W.ctrlpts[1]+2*ot.y*st},ut=D.intersectLine(E.x,E.y,P,S,lt.x,lt.y,0);et?(W.ctrlpts[0]=W.ctrlpts[0]+ot.x*(Ke-Je),W.ctrlpts[1]=W.ctrlpts[1]+ot.y*(Ke-Je)):(W.ctrlpts[0]=ut[0]+ot.x*Ke,W.ctrlpts[1]=ut[1]+ot.y*Ke)}if(Ue||Ze||rt){nt=!0;var it={x:W.ctrlpts[0]-_.x,y:W.ctrlpts[1]-_.y},at=Math.sqrt(it.x*it.x+it.y*it.y),ot={x:it.x/at,y:it.y/at},st=Math.max(P,S),lt={x:W.ctrlpts[0]+2*ot.x*st,y:W.ctrlpts[1]+2*ot.y*st},ct=C.intersectLine(_.x,_.y,k,T,lt.x,lt.y,0);rt?(W.ctrlpts[0]=W.ctrlpts[0]+ot.x*(Ke-tt),W.ctrlpts[1]=W.ctrlpts[1]+ot.y*(Ke-tt)):(W.ctrlpts[0]=ct[0]+ot.x*Ke,W.ctrlpts[1]=ct[1]+ot.y*Ke)}nt&&this.findEndpoints(d)}if("multibezier"===W.edgeType||"bezier"===W.edgeType||"self"===W.edgeType||"compound"===W.edgeType){W.allpts=[],W.allpts.push(W.startX,W.startY);for(var Fe=0;Fe+1=e.desktopTapThreshold2}var M=r(i);b&&(e.hoverData.tapholdCancelled=!0);var N=function(){var t=e.hoverData.dragDelta=e.hoverData.dragDelta||[];0===t.length?(t.push(E[0]),t.push(E[1])):(t[0]+=E[0],t[1]+=E[1])};if(u=!0,t(m,["mousemove","vmousemove","tapdrag"],i,{cyPosition:{x:p[0],y:p[1]}}),3===e.hoverData.which){if(b){var B=new o(i,{type:"cxtdrag",cyPosition:{x:p[0],y:p[1]}});w?w.trigger(B):c.trigger(B),e.hoverData.cxtDragged=!0,e.hoverData.cxtOver&&m===e.hoverData.cxtOver||(e.hoverData.cxtOver&&e.hoverData.cxtOver.trigger(new o(i,{type:"cxtdragout",cyPosition:{x:p[0],y:p[1]}})),e.hoverData.cxtOver=m,m&&m.trigger(new o(i,{type:"cxtdragover",cyPosition:{x:p[0],y:p[1]}})))}}else if(e.hoverData.dragging){if(u=!0,c.panningEnabled()&&c.userPanningEnabled()){var z;if(e.hoverData.justStartedPan){var I=e.hoverData.mdownPos;z={x:(p[0]-I[0])*d,y:(p[1]-I[1])*d},e.hoverData.justStartedPan=!1}else z={x:E[0]*d,y:E[1]*d};c.panBy(z),e.hoverData.dragged=!0}p=e.projectIntoViewport(i.clientX,i.clientY)}else if(1!=g[4]||null!=w&&!w.isEdge()){if(w&&w.isEdge()&&w.active()&&w.unactivate(),w&&w.grabbed()||m==x||(x&&t(x,["mouseout","tapdragout"],i,{cyPosition:{x:p[0],y:p[1]}}),m&&t(m,["mouseover","tapdragover"],i,{ +cyPosition:{x:p[0],y:p[1]}}),e.hoverData.last=m),w&&e.nodeIsDraggable(w))if(b){var L=!e.dragData.didDrag;L&&e.redrawHint("eles",!0),e.dragData.didDrag=!0;var O=[];e.hoverData.draggingEles||y(c.collection(_),{inDragLayer:!0});for(var A=0;A<_.length;A++){var R=_[A];if(e.nodeIsDraggable(R)&&R.grabbed()){var q=R._private.position;if(O.push(R),n.number(E[0])&&n.number(E[1])){var V=!R.isParent();if(V&&(q.x+=E[0],q.y+=E[1]),L){var F=e.hoverData.dragDelta;V&&F&&n.number(F[0])&&n.number(F[1])&&(q.x+=F[0],q.y+=F[1])}}}}e.hoverData.draggingEles=!0;var j=c.collection(O);j.updateCompoundBounds(),j.trigger("position drag"),e.redrawHint("drag",!0),e.redraw()}else N();u=!0}else if(b){if(e.hoverData.dragging||!c.boxSelectionEnabled()||!M&&c.panningEnabled()&&c.userPanningEnabled()){if(!e.hoverData.selecting&&c.panningEnabled()&&c.userPanningEnabled()){var X=s(w,e.hoverData.downs);X&&(e.hoverData.dragging=!0,e.hoverData.justStartedPan=!0,g[4]=0,e.data.bgActivePosistion=a.array2point(f),e.redrawHint("select",!0),e.redraw())}}else e.data.bgActivePosistion=void 0,e.hoverData.selecting||c.trigger("boxstart"),e.hoverData.selecting=!0,e.redrawHint("select",!0),e.redraw();w&&w.isEdge()&&w.active()&&w.unactivate()}return g[2]=p[0],g[3]=p[1],u?(i.stopPropagation&&i.stopPropagation(),i.preventDefault&&i.preventDefault(),!1):void 0}},!1),e.registerBinding(window,"mouseup",function(n){var i=e.hoverData.capture;if(i){e.hoverData.capture=!1;var a=e.cy,s=e.projectIntoViewport(n.clientX,n.clientY),l=e.selection,u=e.findNearestElement(s[0],s[1],!0,!1),c=e.dragData.possibleDragElements,d=e.hoverData.down,h=r(n);if(e.data.bgActivePosistion&&(e.redrawHint("select",!0),e.redraw()),e.hoverData.tapholdCancelled=!0,e.data.bgActivePosistion=void 0,d&&d.unactivate(),3===e.hoverData.which){var p=new o(n,{type:"cxttapend",cyPosition:{x:s[0],y:s[1]}});if(d?d.trigger(p):a.trigger(p),!e.hoverData.cxtDragged){var f=new o(n,{type:"cxttap",cyPosition:{x:s[0],y:s[1]}});d?d.trigger(f):a.trigger(f)}e.hoverData.cxtDragged=!1,e.hoverData.which=null}else if(1===e.hoverData.which){if(null!=d||e.dragData.didDrag||e.hoverData.selecting||e.hoverData.dragged||r(n)||(a.$(function(){return this.selected()}).unselect(),c.length>0&&e.redrawHint("eles",!0),e.dragData.possibleDragElements=c=[]),t(u,["mouseup","tapend","vmouseup"],n,{cyPosition:{x:s[0],y:s[1]}}),e.dragData.didDrag||e.hoverData.dragged||e.hoverData.selecting||t(d,["click","tap","vclick"],n,{cyPosition:{x:s[0],y:s[1]}}),u!=d||e.dragData.didDrag||e.hoverData.selecting||null!=u&&u._private.selectable&&(e.hoverData.dragging||("additive"===a.selectionType()||h?u.selected()?u.unselect():u.select():h||(a.$(":selected").unmerge(u).unselect(),u.select())),e.redrawHint("eles",!0)),e.hoverData.selecting){var v=a.collection(e.getAllInBox(l[0],l[1],l[2],l[3]));e.redrawHint("select",!0),v.length>0&&e.redrawHint("eles",!0),a.trigger("boxend");var g=function(e){return e.selectable()&&!e.selected()};"additive"===a.selectionType()?v.trigger("box").stdFilter(g).select().trigger("boxselect"):(h||a.$(":selected").unmerge(v).unselect(),v.trigger("box").stdFilter(g).select().trigger("boxselect")),e.redraw()}if(e.hoverData.dragging&&(e.hoverData.dragging=!1,e.redrawHint("select",!0),e.redrawHint("eles",!0),e.redraw()),!l[4]){e.redrawHint("drag",!0),e.redrawHint("eles",!0);var y=d&&d.grabbed();b(c),y&&d.trigger("free")}}l[4]=0,e.hoverData.down=null,e.hoverData.cxtStarted=!1,e.hoverData.draggingEles=!1,e.hoverData.selecting=!1,e.dragData.didDrag=!1,e.hoverData.dragged=!1,e.hoverData.dragDelta=[],e.hoverData.mdownPos=null,e.hoverData.mdownGPos=null}},!1);var T=function(t){if(!e.scrollingPage){var r=e.cy,n=e.projectIntoViewport(t.clientX,t.clientY),i=[n[0]*r.zoom()+r.pan().x,n[1]*r.zoom()+r.pan().y];if(e.hoverData.draggingEles||e.hoverData.dragging||e.hoverData.cxtStarted||S())return void t.preventDefault();if(r.panningEnabled()&&r.userPanningEnabled()&&r.zoomingEnabled()&&r.userZoomingEnabled()){t.preventDefault(),e.data.wheelZooming=!0,clearTimeout(e.data.wheelTimeout),e.data.wheelTimeout=setTimeout(function(){e.data.wheelZooming=!1,e.redrawHint("eles",!0),e.redraw()},150);var a;a=null!=t.deltaY?t.deltaY/-250:null!=t.wheelDeltaY?t.wheelDeltaY/1e3:t.wheelDelta/1e3,a*=e.wheelSensitivity;var o=1===t.deltaMode;o&&(a*=33),r.zoom({level:r.zoom()*Math.pow(10,a),renderedPosition:{x:i[0],y:i[1]}})}}};e.registerBinding(e.container,"wheel",T,!0),e.registerBinding(window,"scroll",function(t){e.scrollingPage=!0,clearTimeout(e.scrollingPageTimeout),e.scrollingPageTimeout=setTimeout(function(){e.scrollingPage=!1},250)},!0),e.registerBinding(e.container,"mouseout",function(t){var r=e.projectIntoViewport(t.clientX,t.clientY);e.cy.trigger(new o(t,{type:"mouseout",cyPosition:{x:r[0],y:r[1]}}))},!1),e.registerBinding(e.container,"mouseover",function(t){var r=e.projectIntoViewport(t.clientX,t.clientY);e.cy.trigger(new o(t,{type:"mouseover",cyPosition:{x:r[0],y:r[1]}}))},!1);var D,C,M,N,B,z,I,L,O,A,R,q,V,F,j=function(e,t,r,n){return Math.sqrt((r-e)*(r-e)+(n-t)*(n-t))},X=function(e,t,r,n){return(r-e)*(r-e)+(n-t)*(n-t)};e.registerBinding(e.container,"touchstart",F=function(r){if(k(r)){e.touchData.capture=!0,e.data.bgActivePosistion=void 0;var n=e.cy,i=e.touchData.now,a=e.touchData.earlier;if(r.touches[0]){var s=e.projectIntoViewport(r.touches[0].clientX,r.touches[0].clientY);i[0]=s[0],i[1]=s[1]}if(r.touches[1]){var s=e.projectIntoViewport(r.touches[1].clientX,r.touches[1].clientY);i[2]=s[0],i[3]=s[1]}if(r.touches[2]){var s=e.projectIntoViewport(r.touches[2].clientX,r.touches[2].clientY);i[4]=s[0],i[5]=s[1]}if(r.touches[1]){b(e.dragData.touchDragEles);var l=e.findContainerClientCoords();O=l[0],A=l[1],R=l[2],q=l[3],D=r.touches[0].clientX-O,C=r.touches[0].clientY-A,M=r.touches[1].clientX-O,N=r.touches[1].clientY-A,V=D>=0&&R>=D&&M>=0&&R>=M&&C>=0&&q>=C&&N>=0&&q>=N;var u=n.pan(),c=n.zoom();B=j(D,C,M,N),z=X(D,C,M,N),I=[(D+M)/2,(C+N)/2],L=[(I[0]-u.x)/c,(I[1]-u.y)/c];var d=200,h=d*d;if(h>z&&!r.touches[2]){var f=e.findNearestElement(i[0],i[1],!0,!0),v=e.findNearestElement(i[2],i[3],!0,!0);return f&&f.isNode()?(f.activate().trigger(new o(r,{type:"cxttapstart",cyPosition:{x:i[0],y:i[1]}})),e.touchData.start=f):v&&v.isNode()?(v.activate().trigger(new o(r,{type:"cxttapstart",cyPosition:{x:i[0],y:i[1]}})),e.touchData.start=v):n.trigger(new o(r,{type:"cxttapstart",cyPosition:{x:i[0],y:i[1]}})),e.touchData.start&&(e.touchData.start._private.grabbed=!1),e.touchData.cxt=!0,e.touchData.cxtDragged=!1,e.data.bgActivePosistion=void 0,void e.redraw()}}if(r.touches[2]);else if(r.touches[1]);else if(r.touches[0]){var g=e.findNearestElements(i[0],i[1],!0,!0),x=g[0];if(null!=x&&(x.activate(),e.touchData.start=x,e.touchData.starts=g,e.nodeIsGrabbable(x))){var w=e.dragData.touchDragEles=[];if(e.redrawHint("eles",!0),e.redrawHint("drag",!0),x.selected()){var E=n.$(function(){return this.selected()&&e.nodeIsGrabbable(this)});y(E,{addToList:w})}else m(x,{addToList:w});p(x),x.trigger(new o(r,{type:"grab",cyPosition:{x:i[0],y:i[1]}}))}t(x,["touchstart","tapstart","vmousedown"],r,{cyPosition:{x:i[0],y:i[1]}}),null==x&&(e.data.bgActivePosistion={x:s[0],y:s[1]},e.redrawHint("select",!0),e.redraw()),e.touchData.singleTouchMoved=!1,e.touchData.singleTouchStartTime=+new Date,clearTimeout(e.touchData.tapholdTimeout),e.touchData.tapholdTimeout=setTimeout(function(){e.touchData.singleTouchMoved!==!1||e.pinching||e.touchData.selecting||(t(e.touchData.start,["taphold"],r,{cyPosition:{x:i[0],y:i[1]}}),e.touchData.start||n.$(":selected").unselect())},e.tapholdDuration)}if(r.touches.length>=1){for(var _=e.touchData.startPosition=[],P=0;P=e.touchTapThreshold2}if(i&&e.touchData.cxt){r.preventDefault();var S=r.touches[0].clientX-O,T=r.touches[0].clientY-A,I=r.touches[1].clientX-O,R=r.touches[1].clientY-A,q=X(S,T,I,R),F=q/z,Y=150,W=Y*Y,$=1.5,H=$*$;if(F>=H||q>=W){e.touchData.cxt=!1,e.data.bgActivePosistion=void 0,e.redrawHint("select",!0);var U=new o(r,{type:"cxttapend",cyPosition:{x:c[0],y:c[1]}});e.touchData.start?(e.touchData.start.unactivate().trigger(U),e.touchData.start=null):u.trigger(U)}}if(i&&e.touchData.cxt){var U=new o(r,{type:"cxtdrag",cyPosition:{x:c[0],y:c[1]}});e.data.bgActivePosistion=void 0,e.redrawHint("select",!0),e.touchData.start?e.touchData.start.trigger(U):u.trigger(U),e.touchData.start&&(e.touchData.start._private.grabbed=!1),e.touchData.cxtDragged=!0;var Z=e.findNearestElement(c[0],c[1],!0,!0);e.touchData.cxtOver&&Z===e.touchData.cxtOver||(e.touchData.cxtOver&&e.touchData.cxtOver.trigger(new o(r,{type:"cxtdragout",cyPosition:{x:c[0],y:c[1]}})),e.touchData.cxtOver=Z,Z&&Z.trigger(new o(r,{type:"cxtdragover",cyPosition:{x:c[0],y:c[1]}})))}else if(i&&r.touches[2]&&u.boxSelectionEnabled())r.preventDefault(),e.data.bgActivePosistion=void 0,this.lastThreeTouch=+new Date,e.touchData.selecting||u.trigger("boxstart"),e.touchData.selecting=!0,e.redrawHint("select",!0),l&&0!==l.length&&void 0!==l[0]?(l[2]=(c[0]+c[2]+c[4])/3,l[3]=(c[1]+c[3]+c[5])/3):(l[0]=(c[0]+c[2]+c[4])/3,l[1]=(c[1]+c[3]+c[5])/3,l[2]=(c[0]+c[2]+c[4])/3+1,l[3]=(c[1]+c[3]+c[5])/3+1),l[4]=1,e.touchData.selecting=!0,e.redraw();else if(i&&r.touches[1]&&u.zoomingEnabled()&&u.panningEnabled()&&u.userZoomingEnabled()&&u.userPanningEnabled()){r.preventDefault(),e.data.bgActivePosistion=void 0,e.redrawHint("select",!0);var G=e.dragData.touchDragEles;if(G){e.redrawHint("drag",!0);for(var Q=0;Q=e*e+t*t}}},i.generateRoundRectangle=function(){return this.nodeShapes.roundrectangle={renderer:this,name:"roundrectangle",points:n.generateUnitNgonPointsFitToSquare(4,0),draw:function(e,t,r,n,i){this.renderer.nodeShapeImpl(this.name,e,t,r,n,i)},intersectLine:function(e,t,r,i,a,o,s){return n.roundRectangleIntersectLine(a,o,e,t,r,i,s)},checkPoint:function(e,t,r,i,a,o,s){var l=n.getRoundRectangleRadius(i,a);if(n.pointInsidePolygon(e,t,this.points,o,s,i,a-2*l,[0,-1],r))return!0;if(n.pointInsidePolygon(e,t,this.points,o,s,i-2*l,a,[0,-1],r))return!0;var u=function(e,t,r,n,i,a,o){return e-=r,t-=n,e/=i/2+o,t/=a/2+o,1>=e*e+t*t};return u(e,t,o-i/2+l,s-a/2+l,2*l,2*l,r)?!0:u(e,t,o+i/2-l,s-a/2+l,2*l,2*l,r)?!0:u(e,t,o+i/2-l,s+a/2-l,2*l,2*l,r)?!0:!!u(e,t,o-i/2+l,s+a/2-l,2*l,2*l,r)}}},i.registerNodeShapes=function(){var e=this.nodeShapes={},t=this;this.generateEllipse(),this.generatePolygon("triangle",n.generateUnitNgonPointsFitToSquare(3,0)),this.generatePolygon("rectangle",n.generateUnitNgonPointsFitToSquare(4,0)),e.square=e.rectangle,this.generateRoundRectangle(),this.generatePolygon("diamond",[0,1,1,0,0,-1,-1,0]),this.generatePolygon("pentagon",n.generateUnitNgonPointsFitToSquare(5,0)),this.generatePolygon("hexagon",n.generateUnitNgonPointsFitToSquare(6,0)),this.generatePolygon("heptagon",n.generateUnitNgonPointsFitToSquare(7,0)),this.generatePolygon("octagon",n.generateUnitNgonPointsFitToSquare(8,0));var r=new Array(20),i=n.generateUnitNgonPoints(5,0),a=n.generateUnitNgonPoints(5,Math.PI/5),o=.5*(3-Math.sqrt(5));o*=1.57;for(var s=0;ss;s++)r[4*s]=i[2*s],r[4*s+1]=i[2*s+1],r[4*s+2]=a[2*s],r[4*s+3]=a[2*s+1];r=n.fitPolygonToSquare(r),this.generatePolygon("star",r),this.generatePolygon("vee",[-1,-1,0,-.333,1,-1,0,1]),this.generatePolygon("rhomboid",[-1,-1,.333,-1,1,1,-.333,1]),e.makePolygon=function(e){var r,n=e.join("$"),i="polygon-"+n;return(r=this[i])?r:t.generatePolygon(i,e)}},t.exports=i},{"../../../math":85}],63:[function(e,t,r){"use strict";var n=e("../../../util"),i={};i.timeToRender=function(){return this.redrawTotalTime/this.redrawCount},i.redraw=function(e){e=e||n.staticEmptyObject();var t=this;void 0===t.averageRedrawTime&&(t.averageRedrawTime=0),void 0===t.lastRedrawTime&&(t.lastRedrawTime=0),void 0===t.lastDrawTime&&(t.lastDrawTime=0),t.requestedFrame=!0,t.renderOptions=e},i.beforeRender=function(e,t){if(!this.destroyed){t=t||0;var r=this.beforeRenderCallbacks;r.push({fn:e,priority:t}),r.sort(function(e,t){return t.priority-e.priority})}};var a=function(e,t,r){for(var n=e.beforeRenderCallbacks,i=0;io)},o.drawElementText=function(e,t,r){var n=this;if(void 0===r){if(!n.eleTextBiggerThanMin(t))return}else if(!r)return;if(t.isNode()){var i=t.pstyle("label");if(!i||!i.value)return;var a=t.pstyle("text-halign").strValue;t.pstyle("text-valign").strValue;switch(a){case"left":e.textAlign="right";break;case"right":e.textAlign="left";break;default:e.textAlign="center"}e.textBaseline="bottom"}else{var i=t.pstyle("label"),o=t.pstyle("source-label"),s=t.pstyle("target-label");if(!(i&&i.value||o&&o.value||s&&s.value))return;e.textAlign="center",e.textBaseline="bottom"}n.drawText(e,t),t.isEdge()&&(n.drawText(e,t,"source"),n.drawText(e,t,"target"))},o.drawNodeText=o.drawEdgeText=o.drawElementText,o.getFontCache=function(e){var t;this.fontCaches=this.fontCaches||[];for(var r=0;r0||k>0&&S>0){var T=l;switch(m){case"left":T-=h;break;case"center":T-=h/2;break;case"right":}var D=u-p;if(P>0){var C=e.fillStyle,M=t.pstyle("text-background-color").value;e.fillStyle="rgba("+M[0]+","+M[1]+","+M[2]+","+P*s+")";var N=t.pstyle("text-background-shape").strValue;"roundrectangle"==N?n(e,T,D,h,p,2):e.fillRect(T,D,h,p),e.fillStyle=C}if(k>0&&S>0){var B=e.strokeStyle,z=e.lineWidth,I=t.pstyle("text-border-color").value,L=t.pstyle("text-border-style").value;if(e.strokeStyle="rgba("+I[0]+","+I[1]+","+I[2]+","+S*s+")",e.lineWidth=k,e.setLineDash)switch(L){case"dotted":e.setLineDash([1,1]);break;case"dashed":e.setLineDash([4,2]); +break;case"double":e.lineWidth=k/4,e.setLineDash([]);break;case"solid":e.setLineDash([])}if(e.strokeRect(T,D,h,p),"double"===L){var O=k/2;e.strokeRect(T+O,D+O,h-2*O,p-2*O)}e.setLineDash&&e.setLineDash([]),e.lineWidth=z,e.strokeStyle=B}}var A=2*t.pstyle("text-outline-width").pfValue;if(A>0&&(e.lineWidth=A),"wrap"===t.pstyle("text-wrap").value){var R=i.getPrefixedProperty(o,"labelWrapCachedLines",r),q=p/R.length;switch(b){case"top":u-=(R.length-1)*q;break;case"center":case"bottom":u-=(R.length-1)*q}for(var V=0;V0&&e.strokeText(R[V],l,u),e.fillText(R[V],l,u),u+=q}else A>0&&e.strokeText(c,l,u),e.fillText(c,l,u);0!==x&&(e.rotate(-x),e.translate(-E,-_)),this.shadowStyle(e,"transparent",0)}}},t.exports=o},{"../../../math":85,"../../../util":100}],69:[function(e,t,r){"use strict";var n=e("../../../is"),i={};i.drawNode=function(e,t,r,i){var a,o,s=this,l=t._private.rscratch,u=t._private,c=c||u.position;if(n.number(c.x)&&n.number(c.y)&&t.visible()){var d,h=t.effectiveOpacity(),p=this.usePaths(),f=!1,v=t.pstyle("padding").pfValue;a=t.width()+2*v,o=t.height()+2*v,e.lineWidth=t.pstyle("border-width").pfValue;var g;r&&(g=r,e.translate(-g.x1,-g.y1));var y,m=t.pstyle("background-image"),b=m.value[2]||m.value[1];if(void 0!==b){var x=t.pstyle("background-image-crossorigin");y=this.getCachedImage(b,x,function(){t.trigger("background"),s.redrawHint("eles",!0),s.redrawHint("drag",!0),s.drawingImage=!0,s.redraw()});var w=u.backgrounding;u.backgrounding=!y.complete,w!==u.backgrounding&&t.updateStyle(!1)}var E=t.pstyle("background-color").value,_=t.pstyle("border-color").value,P=t.pstyle("border-style").value;this.fillStyle(e,E[0],E[1],E[2],t.pstyle("background-opacity").value*h),this.strokeStyle(e,_[0],_[1],_[2],t.pstyle("border-opacity").value*h);var S=t.pstyle("shadow-blur").pfValue,k=t.pstyle("shadow-opacity").value,T=t.pstyle("shadow-color").value,D=t.pstyle("shadow-offset-x").pfValue,C=t.pstyle("shadow-offset-y").pfValue;if(this.shadowStyle(e,T,k,S,D,C),e.lineJoin="miter",e.setLineDash)switch(P){case"dotted":e.setLineDash([1,1]);break;case"dashed":e.setLineDash([4,2]);break;case"solid":case"double":e.setLineDash([])}var M=t.pstyle("shape").strValue,N=t.pstyle("shape-polygon-points").pfValue;if(p){var B=M+"$"+a+"$"+o+("polygon"===M?"$"+N.join("$"):"");e.translate(c.x,c.y),l.pathCacheKey===B?(d=l.pathCache,f=!0):(d=new Path2D,l.pathCacheKey=B,l.pathCache=d)}if(!f){var z=c;p&&(z={x:0,y:0}),s.nodeShapes[this.getNodeShape(t)].draw(d||e,z.x,z.y,a,o)}p?e.fill(d):e.fill(),this.shadowStyle(e,"transparent",0),void 0!==b&&y.complete&&this.drawInscribedImage(e,y,t);var I=t.pstyle("background-blacken").value,L=t.pstyle("border-width").pfValue;if(this.hasPie(t)&&(this.drawPie(e,t,h),0===I&&0===L||p||s.nodeShapes[this.getNodeShape(t)].draw(e,c.x,c.y,a,o)),I>0?(this.fillStyle(e,0,0,0,I),p?e.fill(d):e.fill()):0>I&&(this.fillStyle(e,255,255,255,-I),p?e.fill(d):e.fill()),L>0&&(p?e.stroke(d):e.stroke(),"double"===P)){e.lineWidth=t.pstyle("border-width").pfValue/3;var O=e.globalCompositeOperation;e.globalCompositeOperation="destination-out",p?e.stroke(d):e.stroke(),e.globalCompositeOperation=O}p&&e.translate(-c.x,-c.y),e.setLineDash&&e.setLineDash([]),s.drawElementText(e,t,i);var A=t.pstyle("overlay-padding").pfValue,R=t.pstyle("overlay-opacity").value,q=t.pstyle("overlay-color").value;R>0&&(this.fillStyle(e,q[0],q[1],q[2],R),s.nodeShapes.roundrectangle.draw(e,t._private.position.x,t._private.position.y,a+2*A,o+2*A),e.fill()),r&&e.translate(g.x1,g.y1)}},i.hasPie=function(e){return e=e[0],e._private.hasPie},i.drawPie=function(e,t,r,n){t=t[0];var i=t._private,a=t.cy().style(),o=t.pstyle("pie-size"),s=t.width(),l=t.height(),n=n||i.position,u=n.x,c=n.y,d=Math.min(s,l)/2,h=0,p=this.usePaths();p&&(u=0,c=0),"%"===o.units?d=d*o.value/100:void 0!==o.pfValue&&(d=o.pfValue/2);for(var f=1;f<=a.pieBackgroundN;f++){var v=t.pstyle("pie-"+f+"-background-size").value,g=t.pstyle("pie-"+f+"-background-color").value,y=t.pstyle("pie-"+f+"-background-opacity").value*r,m=v/100;m+h>1&&(m=1-h);var b=1.5*Math.PI+2*Math.PI*h,x=2*Math.PI*m,w=b+x;0===v||h>=1||h+m>1||(e.beginPath(),e.moveTo(u,c),e.arc(u,c,d,b,w),e.closePath(),this.fillStyle(e,g[0],g[1],g[2],y),e.fill(),h+=m)}},t.exports=i},{"../../../is":83}],70:[function(e,t,r){"use strict";var n={},i=e("../../../util"),a=100;n.getPixelRatio=function(){var e=this.data.contexts[0];if(null!=this.forcedPixelRatio)return this.forcedPixelRatio;var t=e.backingStorePixelRatio||e.webkitBackingStorePixelRatio||e.mozBackingStorePixelRatio||e.msBackingStorePixelRatio||e.oBackingStorePixelRatio||e.backingStorePixelRatio||1;return(window.devicePixelRatio||1)/t},n.paintCache=function(e){for(var t,r=this.paintCaches=this.paintCaches||[],n=!0,i=0;i0?(e.shadowBlur=n*o,e.shadowColor="rgba("+t[0]+","+t[1]+","+t[2]+","+r+")",e.shadowOffsetX=i*o,e.shadowOffsetY=a*o):(e.shadowBlur=0,e.shadowColor="transparent",e.shadowOffsetX=0,e.shadowOffsetY=0)},n.matchCanvasSize=function(e){var t=this,r=t.data,n=t.findContainerClientCoords(),i=n[2],a=n[3],o=t.getPixelRatio(),s=t.motionBlurPxRatio;e!==t.data.bufferCanvases[t.MOTIONBLUR_BUFFER_NODE]&&e!==t.data.bufferCanvases[t.MOTIONBLUR_BUFFER_DRAG]||(o=s);var l,u=i*o,c=a*o;if(u!==t.canvasWidth||c!==t.canvasHeight){t.fontCaches=null;var d=r.canvasContainer;d.style.width=i+"px",d.style.height=a+"px";for(var h=0;h=o&&(l=r.bufferCanvases[t.TEXTURE_BUFFER],t.textureMult=2,l.width=u*t.textureMult,l.height=c*t.textureMult),t.canvasWidth=u,t.canvasHeight=c}},n.renderTo=function(e,t,r,n){this.render({forcedContext:e,forcedZoom:t,forcedPan:r,drawAllLayers:!0,forcedPxRatio:n})},n.render=function(e){function t(e,t,r,n,i){var a=e.globalCompositeOperation;e.globalCompositeOperation="destination-out",c.fillStyle(e,255,255,255,c.motionBlurTransparency),e.fillRect(t,r,n,i),e.globalCompositeOperation=a}function r(e,r){var i,a,s,d;c.clearingMotionBlur||e!==p.bufferContexts[c.MOTIONBLUR_BUFFER_NODE]&&e!==p.bufferContexts[c.MOTIONBLUR_BUFFER_DRAG]?(i=k,a=P,s=c.canvasWidth,d=c.canvasHeight):(i={x:S.x*y,y:S.y*y},a=_*y,s=c.canvasWidth*y,d=c.canvasHeight*y),e.setTransform(1,0,0,1,0,0),"motionBlur"===r?t(e,0,0,s,d):n||void 0!==r&&!r||e.clearRect(0,0,s,d),o||(e.translate(i.x,i.y),e.scale(a,a)),u&&e.translate(u.x,u.y),l&&e.scale(l,l)}e=e||i.staticEmptyObject();var n=e.forcedContext,o=e.drawAllLayers,s=e.drawOnlyNodeLayer,l=e.forcedZoom,u=e.forcedPan,c=this,d=void 0===e.forcedPxRatio?this.getPixelRatio():e.forcedPxRatio,h=c.cy,p=c.data,f=p.canvasNeedsRedraw,v=c.textureOnViewport&&!n&&(c.pinching||c.hoverData.dragging||c.swipePanning||c.data.wheelZooming),g=void 0!==e.motionBlur?e.motionBlur:c.motionBlur,y=c.motionBlurPxRatio,m=h.hasCompoundNodes(),b=c.hoverData.draggingEles,x=!(!c.hoverData.selecting&&!c.touchData.selecting);g=g&&!n&&c.motionBlurEnabled&&!x;var w=g;n||(c.prevPxRatio!==d&&(c.invalidateContainerClientCoordsCache(),c.matchCanvasSize(c.container),c.redrawHint("eles",!0),c.redrawHint("drag",!0)),c.prevPxRatio=d),!n&&c.motionBlurTimeout&&clearTimeout(c.motionBlurTimeout),g&&(null==c.mbFrames&&(c.mbFrames=0),c.drawingImage||c.mbFrames++,c.mbFrames<3&&(w=!1),c.mbFrames>c.minMbLowQualFrames&&(c.motionBlurPxRatio=c.mbPxRBlurry)),c.clearingMotionBlur&&(c.motionBlurPxRatio=1),c.textureDrawLastFrame&&!v&&(f[c.NODE]=!0,f[c.SELECT_BOX]=!0);var E=h.style()._private.coreStyle,_=h.zoom(),P=void 0!==l?l:_,S=h.pan(),k={x:S.x,y:S.y},T={zoom:_,pan:{x:S.x,y:S.y}},D=c.prevViewport,C=void 0===D||T.zoom!==D.zoom||T.pan.x!==D.pan.x||T.pan.y!==D.pan.y;C||b&&!m||(c.motionBlurPxRatio=1),u&&(k=u),P*=d,k.x*=d,k.y*=d;var M=c.getCachedZSortedEles();if(v||(c.textureDrawLastFrame=!1),v){c.textureDrawLastFrame=!0;var N;if(!c.textureCache){c.textureCache={},N=c.textureCache.bb=h.mutableElements().boundingBox(),c.textureCache.texture=c.data.bufferCanvases[c.TEXTURE_BUFFER];var B=c.data.bufferContexts[c.TEXTURE_BUFFER];B.setTransform(1,0,0,1,0,0),B.clearRect(0,0,c.canvasWidth*c.textureMult,c.canvasHeight*c.textureMult),c.render({forcedContext:B,drawOnlyNodeLayer:!0,forcedPxRatio:d*c.textureMult});var T=c.textureCache.viewport={zoom:h.zoom(),pan:h.pan(),width:c.canvasWidth,height:c.canvasHeight};T.mpan={x:(0-T.pan.x)/T.zoom,y:(0-T.pan.y)/T.zoom}}f[c.DRAG]=!1,f[c.NODE]=!1;var z=p.contexts[c.NODE],I=c.textureCache.texture,T=c.textureCache.viewport;N=c.textureCache.bb,z.setTransform(1,0,0,1,0,0),g?t(z,0,0,T.width,T.height):z.clearRect(0,0,T.width,T.height);var L=E["outside-texture-bg-color"].value,O=E["outside-texture-bg-opacity"].value;c.fillStyle(z,L[0],L[1],L[2],O),z.fillRect(0,0,T.width,T.height);var _=h.zoom();r(z,!1),z.clearRect(T.mpan.x,T.mpan.y,T.width/T.zoom/d,T.height/T.zoom/d),z.drawImage(I,T.mpan.x,T.mpan.y,T.width/T.zoom/d,T.height/T.zoom/d)}else c.textureOnViewport&&!n&&(c.textureCache=null);var A=h.extent(),R=c.pinching||c.hoverData.dragging||c.swipePanning||c.data.wheelZooming||c.hoverData.draggingEles,q=c.hideEdgesOnViewport&&R,V=[];if(V[c.NODE]=!f[c.NODE]&&g&&!c.clearedForMotionBlur[c.NODE]||c.clearingMotionBlur,V[c.NODE]&&(c.clearedForMotionBlur[c.NODE]=!0),V[c.DRAG]=!f[c.DRAG]&&g&&!c.clearedForMotionBlur[c.DRAG]||c.clearingMotionBlur,V[c.DRAG]&&(c.clearedForMotionBlur[c.DRAG]=!0),f[c.NODE]||o||s||V[c.NODE]){var F=g&&!V[c.NODE]&&1!==y,z=n||(F?c.data.bufferContexts[c.MOTIONBLUR_BUFFER_NODE]:p.contexts[c.NODE]),j=g&&!F?"motionBlur":void 0;r(z,j),q?c.drawCachedNodes(z,M.nondrag,d,A):c.drawLayeredElements(z,M.nondrag,d,A),o||g||(f[c.NODE]=!1)}if(!s&&(f[c.DRAG]||o||V[c.DRAG])){var F=g&&!V[c.DRAG]&&1!==y,z=n||(F?c.data.bufferContexts[c.MOTIONBLUR_BUFFER_DRAG]:p.contexts[c.DRAG]);r(z,g&&!F?"motionBlur":void 0),q?c.drawCachedNodes(z,M.drag,d,A):c.drawCachedElements(z,M.drag,d,A),o||g||(f[c.DRAG]=!1)}if(c.showFps||!s&&f[c.SELECT_BOX]&&!o){var z=n||p.contexts[c.SELECT_BOX];if(r(z),1==c.selection[4]&&(c.hoverData.selecting||c.touchData.selecting)){var _=c.cy.zoom(),X=E["selection-box-border-width"].value/_;z.lineWidth=X,z.fillStyle="rgba("+E["selection-box-color"].value[0]+","+E["selection-box-color"].value[1]+","+E["selection-box-color"].value[2]+","+E["selection-box-opacity"].value+")",z.fillRect(c.selection[0],c.selection[1],c.selection[2]-c.selection[0],c.selection[3]-c.selection[1]),X>0&&(z.strokeStyle="rgba("+E["selection-box-border-color"].value[0]+","+E["selection-box-border-color"].value[1]+","+E["selection-box-border-color"].value[2]+","+E["selection-box-opacity"].value+")",z.strokeRect(c.selection[0],c.selection[1],c.selection[2]-c.selection[0],c.selection[3]-c.selection[1]))}if(p.bgActivePosistion&&!c.hoverData.selecting){var _=c.cy.zoom(),Y=p.bgActivePosistion;z.fillStyle="rgba("+E["active-bg-color"].value[0]+","+E["active-bg-color"].value[1]+","+E["active-bg-color"].value[2]+","+E["active-bg-opacity"].value+")",z.beginPath(),z.arc(Y.x,Y.y,E["active-bg-size"].pfValue/_,0,2*Math.PI),z.fill()}var W=c.lastRedrawTime;if(c.showFps&&W){W=Math.round(W);var $=Math.round(1e3/W);z.setTransform(1,0,0,1,0,0),z.fillStyle="rgba(255, 0, 0, 0.75)",z.strokeStyle="rgba(255, 0, 0, 0.75)",z.lineWidth=1,z.fillText("1 frame = "+W+" ms = "+$+" fps",0,20);var H=60;z.strokeRect(0,30,250,20),z.fillRect(0,30,250*Math.min($/H,1),20)}o||(f[c.SELECT_BOX]=!1)}if(g&&1!==y){var U=p.contexts[c.NODE],Z=c.data.bufferCanvases[c.MOTIONBLUR_BUFFER_NODE],G=p.contexts[c.DRAG],Q=c.data.bufferCanvases[c.MOTIONBLUR_BUFFER_DRAG],K=function(e,r,n){e.setTransform(1,0,0,1,0,0),n||!w?e.clearRect(0,0,c.canvasWidth,c.canvasHeight):t(e,0,0,c.canvasWidth,c.canvasHeight);var i=y;e.drawImage(r,0,0,c.canvasWidth*i,c.canvasHeight*i,0,0,c.canvasWidth,c.canvasHeight)};(f[c.NODE]||V[c.NODE])&&(K(U,Z,V[c.NODE]),f[c.NODE]=!1),(f[c.DRAG]||V[c.DRAG])&&(K(G,Q,V[c.DRAG]),f[c.DRAG]=!1)}c.prevViewport=T,c.clearingMotionBlur&&(c.clearingMotionBlur=!1,c.motionBlurCleared=!0,c.motionBlur=!0),g&&(c.motionBlurTimeout=setTimeout(function(){c.motionBlurTimeout=null,c.clearedForMotionBlur[c.NODE]=!1,c.clearedForMotionBlur[c.DRAG]=!1,c.motionBlur=!1,c.clearingMotionBlur=!v,c.mbFrames=0,f[c.NODE]=!0,f[c.DRAG]=!0,c.redraw()},a)),c.drawingImage=!1,n||c.initrender||(c.initrender=!0,h.trigger("initrender")),n||h.trigger("render")},t.exports=n},{"../../../util":100}],71:[function(e,t,r){"use strict";var n=e("../../../math"),i={};i.drawPolygonPath=function(e,t,r,n,i,a){var o=n/2,s=i/2;e.beginPath&&e.beginPath(),e.moveTo(t+o*a[0],r+s*a[1]);for(var l=1;li)i=u;else if(y>=d||i>c)return null;var m=Math.pow(2,i),w=t.h*m,E=t.w*m,_=g.imgCaches=g.imgCaches||{},P=_[i];if(P)return P;var S;if(S=s>=w?s:l>=w?l:Math.ceil(w/l)*l,w>v||E>f||!b&&e.isEdge()||!x&&e.isParent())return null;var k=o.getTextureQueue(S),D=k[k.length-2],C=function(){return o.recycleTexture(S,E)||o.addTexture(S,E)};D||(D=k[k.length-1]),D||(D=C()),D.width-D.usedWidth=O;O++){var A=_[O];if(A){M=A;break}}var R=M&&M.level===i+1?M:null,q=function(){D.context.drawImage(R.texture.canvas,R.x,0,R.width,R.height,D.usedWidth,0,E,w)};if(B(R))q();else if(B(M)){if(!I)return o.queueElement(e,t,M.level-1),M;for(var O=M.level;O>i;O--)R=o.getElement(e,t,r,O,T.downscale);q()}else{var V;if(!z&&!I&&!L)for(var O=i-1;O>=u;O--){var A=_[O];if(A){V=A;break}}if(B(V))return o.queueElement(e,t,i),V;D.context.translate(D.usedWidth,0),D.context.scale(m,m),p.drawElement(D.context,e,t,N),D.context.scale(1/m,1/m),D.context.translate(-D.usedWidth,0)}return P=_[i]={ele:e,x:D.usedWidth,texture:D,level:i,scale:m,width:E,height:w,scaledLabelShown:N},D.usedWidth+=Math.ceil(E+h),D.eleCaches.push(P),o.checkTextureFullness(D),P},C.invalidateElement=function(e){var t=this,r=e._private.rscratch.imgCaches;if(r)for(var n=u;c>=n;n++){var a=r[n];if(a){var o=a.texture;o.invalidatedWidth+=a.width,r[n]=null,i.removeFromArray(o.eleCaches,a),t.checkTextureUtility(o)}}},C.checkTextureUtility=function(e){e.invalidatedWidth>=g*e.width&&this.retireTexture(e)},C.checkTextureFullness=function(e){var t=this,r=t.getTextureQueue(e.height);e.usedWidth/e.width>y&&e.fullnessChecks>=m?i.removeFromArray(r,e):e.fullnessChecks++},C.retireTexture=function(e){var t=this,r=e.height,n=t.getTextureQueue(r);i.removeFromArray(n,e),e.retired=!0;for(var a=e.eleCaches,o=0;o=t)return s.retired=!1,s.usedWidth=0,s.invalidatedWidth=0,s.fullnessChecks=0,i.clearArray(s.eleCaches),s.context.clearRect(0,0,s.width,s.height),i.removeFromArray(a,s),n.push(s),s}},C.queueElement=function(e,t,r){var i=this,a=i.getElementQueue(),o=i.getElementIdToQueue(),s=e.id(),l=o[s];if(l)l.level=Math.max(l.level,r),l.reqs++,a.updateItem(l);else{var u={ele:e,bb:t,position:n.copyPosition(e.position()),level:r,reqs:1};e.isEdge()&&(u.positions={source:n.copyPosition(e.source().position()),target:n.copyPosition(e.target().position())}),a.push(u),o[s]=u}},C.dequeue=function(e,t){for(var r=this,i=r.getElementQueue(),a=r.getElementIdToQueue(),o=[],s=0;k>s&&i.size()>0;s++){var l=i.pop();a[l.ele.id()]=null,o.push(l);var u,c=l.ele;u=(!c.isEdge()||n.arePositionsSame(c.source().position(),l.positions.source)&&n.arePositionsSame(c.target().position(),l.positions.target))&&n.arePositionsSame(c.position(),l.position)?l.bb:c.boundingBox(),r.getElement(l.ele,u,e,l.level,T.dequeue)}return o},C.onDequeue=function(e){this.onDequeues.push(e)},C.offDequeue=function(e){i.removeFromArray(this.onDequeues,e)},C.setupDequeueing=o.setupDequeueing({deqRedrawThreshold:S,deqCost:w,deqAvgCost:E,deqNoDrawCost:_,deqFastCost:P,deq:function(e,t,r){return e.dequeue(t,r)},onDeqd:function(e,t){for(var r=0;r0&&s>0){f.clearRect(0,0,o,s),e.bg&&(f.fillStyle=e.bg,f.rect(0,0,o,s),f.fill()),f.globalCompositeOperation="source-over";var v=this.getCachedZSortedEles();if(e.full)f.translate(-i.x1*c,-i.y1*c),f.scale(c,c),this.drawElements(f,v);else{var g=t.pan(),y={x:g.x*c,y:g.y*c};c*=t.zoom(),f.translate(y.x,y.y),f.scale(c,c),this.drawElements(f,v)}}return p},i.png=function(e){return this.bufferCanvasImage(e).toDataURL("image/png")},i.jpg=function(e){return this.bufferCanvasImage(e).toDataURL("image/jpeg")},t.exports=i},{"../../../is":83}],74:[function(e,t,r){"use strict";function n(e){var t=this;t.data={canvases:new Array(u.CANVAS_LAYERS),contexts:new Array(u.CANVAS_LAYERS),canvasNeedsRedraw:new Array(u.CANVAS_LAYERS),bufferCanvases:new Array(u.BUFFER_COUNT),bufferContexts:new Array(u.CANVAS_LAYERS)},t.data.canvasContainer=document.createElement("div");var r=t.data.canvasContainer.style;t.data.canvasContainer.setAttribute("style","-webkit-tap-highlight-color: rgba(0,0,0,0);"),r.position="relative",r.zIndex="0",r.overflow="hidden";var n=e.cy.container();n.appendChild(t.data.canvasContainer),n.setAttribute("style",(n.getAttribute("style")||"")+"-webkit-tap-highlight-color: rgba(0,0,0,0);");for(var i=0;i0&&t.data.lyrTxrCache.invalidateElements(r)})}var i=e("../../../util"),a=e("../../../is"),o=e("./ele-texture-cache"),s=e("./layered-texture-cache"),l=n,u=n.prototype;u.CANVAS_LAYERS=3,u.SELECT_BOX=0,u.DRAG=1,u.NODE=2,u.BUFFER_COUNT=3,u.TEXTURE_BUFFER=0,u.MOTIONBLUR_BUFFER_NODE=1,u.MOTIONBLUR_BUFFER_DRAG=2,u.redrawHint=function(e,t){var r=this;switch(e){case"eles":r.data.canvasNeedsRedraw[u.NODE]=t;break;case"drag":r.data.canvasNeedsRedraw[u.DRAG]=t;break;case"select":r.data.canvasNeedsRedraw[u.SELECT_BOX]=t}};var c="undefined"!=typeof Path2D;u.path2dEnabled=function(e){return void 0===e?this.pathsEnabled:void(this.pathsEnabled=!!e)},u.usePaths=function(){return c&&this.pathsEnabled},[e("./arrow-shapes"),e("./drawing-elements"),e("./drawing-edges"),e("./drawing-images"),e("./drawing-label-text"),e("./drawing-nodes"),e("./drawing-redraw"),e("./drawing-shapes"),e("./export-image"),e("./node-shapes")].forEach(function(e){i.extend(u,e)}),t.exports=l},{"../../../is":83,"../../../util":100,"./arrow-shapes":64,"./drawing-edges":65,"./drawing-elements":66,"./drawing-images":67,"./drawing-label-text":68,"./drawing-nodes":69,"./drawing-redraw":70,"./drawing-shapes":71,"./ele-texture-cache":72,"./export-image":73,"./layered-texture-cache":75,"./node-shapes":76}],75:[function(e,t,r){"use strict";function n(e,t){null!=e.imageSmoothingEnabled?e.imageSmoothingEnabled=t:(e.webkitImageSmoothingEnabled=t,e.mozImageSmoothingEnabled=t,e.msImageSmoothingEnabled=t)}var i=e("../../../util"),a=e("../../../math"),o=e("../../../heap"),s=e("../../../is"),l=e("./texture-cache-defs"),u=1,c=-4,d=2,h=3.99,p=50,f=50,v=!0,g=.15,y=.1,m=.9,b=.9,x=1,w=250,E=16e6,_=!0,P=!0,S=!0,k=function(e,t){var r=this,n=r.renderer=e;r.layersByLevel={},r.firstGet=!0,r.lastInvalidationTime=i.performanceNow()-2*w,r.skipping=!1,n.beforeRender(function(e,t){t-r.lastInvalidationTime<=w?r.skipping=!0:r.skipping=!1});var a=function(e,t){return t.reqs-e.reqs};r.layersQueue=new o(a),r.eleTxrCache=t,r.setupEleCacheInvalidation(),r.setupDequeueing()},T=k.prototype,D=0,C=Math.pow(2,53)-1;T.makeLayer=function(e,t){var r=Math.pow(2,t),n=Math.ceil(e.w*r),i=Math.ceil(e.h*r),a=document.createElement("canvas");a.width=n,a.height=i;var o={id:D=++D%C,bb:e,level:t,width:n,height:i,canvas:a,context:a.getContext("2d"),eles:[],elesQueue:[],reqs:0},s=o.context,l=-o.bb.x1,u=-o.bb.y1;return s.scale(r,r),s.translate(l,u),o},T.getLayers=function(e,t,r){var n=this,o=n.renderer,s=o.cy,l=s.zoom(),p=n.firstGet;if(n.firstGet=!1,null==r)if(r=Math.ceil(a.log2(l*t)),c>r)r=c;else if(l>=h||r>d)return null;n.validateLayersElesOrdering(r,e);var f,v,g=n.layersByLevel,y=Math.pow(2,r),m=g[r]=g[r]||[],b=n.levelIsComplete(r,e),x=function(){var t=function(t){return n.validateLayersElesOrdering(t,e),n.levelIsComplete(t,e)?(v=g[t],!0):void 0},a=function(e){if(!v)for(var n=r+e;n>=c&&d>=n&&!t(n);n+=e);};a(1),a(-1);for(var o=m.length-1;o>=0;o--){var s=m[o];s.invalid&&i.removeFromArray(m,s)}};if(b)return m;x();var w=function(){if(!f){f=a.makeBoundingBox();for(var t=0;tE)return null;var a=n.makeLayer(f,r);if(null!=t){var o=m.indexOf(t)+1;m.splice(o,0,a)}else(void 0===e.insert||e.insert)&&m.unshift(a);return a};if(n.skipping&&!p)return null;for(var S=null,k=e.length/u,T=_&&!p,D=0;D=k||!a.boundingBoxInBoundingBox(S.bb,C.boundingBox()))&&(S=P({insert:!0,after:S}),!S))return null;v||T?n.queueLayer(S,C):n.drawEleInLayer(S,C,r,t),S.eles.push(C),N[r]=S}}return v?v:T?null:m},T.getEleLevelForLayerLevel=function(e,t){return e},T.drawEleInLayer=function(e,t,r,i){var a=this,o=this.renderer,s=e.context,l=t.boundingBox();if(0!==l.w&&0!==l.h){var u=a.eleTxrCache,c=P?u.reasons.highQuality:void 0;r=a.getEleLevelForLayerLevel(r,i);var d=S?u.getElement(t,l,null,r,c):null;d?(v&&n(s,!1),s.drawImage(d.texture.canvas,d.x,0,d.width,d.height,l.x1,l.y1,l.w,l.h),v&&n(s,!0)):o.drawElement(s,t)}},T.levelIsComplete=function(e,t){var r=this,n=r.layersByLevel[e];if(!n||0===n.length)return!1;for(var i=0,a=0;a0)return!1;if(o.invalid)return!1;i+=o.eles.length}return i===t.length},T.validateLayersElesOrdering=function(e,t){var r=this.layersByLevel[e];if(r)for(var n=0;na)this.invalidateLayer(i);else for(var s=a,o=0;o=h;h++){var p=u[h];p&&(a&&r.getEleLevelForLayerLevel(p.level)!==a.level||t(p,o,a))}},T.haveLayers=function(){for(var e=this,t=!1,r=c;d>=r;r++){var n=e.layersByLevel[r];if(n&&n.length>0){t=!0;break}}return t},T.invalidateElements=function(e){var t=this;t.lastInvalidationTime=i.performanceNow(),0!==e.length&&t.haveLayers()&&t.updateElementsInLayers(e,function(e,r,n){t.invalidateLayer(e)})},T.invalidateLayer=function(e){if(this.lastInvalidationTime=i.performanceNow(),!e.invalid){var t=e.level,r=e.eles,n=this.layersByLevel[t];i.removeFromArray(n,e),e.elesQueue=[],e.invalid=!0,e.replacement&&(e.replacement.invalid=!0);for(var a=0;ai&&0!==r.size();){var a=r.peek();if(a.replacement)r.pop();else if(a.replaces&&a!==a.replaces.replacement)r.pop();else if(a.invalid)r.pop();else{var o=a.elesQueue.shift();o&&(t.drawEleInLayer(a,o,a.level,e),i++),0===n.length&&n.push(!0),0===a.elesQueue.length&&(r.pop(),a.reqs=0,a.replaces&&t.applyLayerReplacement(a),t.requestRedraw())}}return n},T.applyLayerReplacement=function(e){var t=this,r=t.layersByLevel[e.level],n=e.replaces,i=r.indexOf(n);if(!(0>i||n.invalid)){r[i]=e;for(var a=0;ac){var y=i-(o?u:0);if(g>=e.deqFastCost*y)break}else if(o){if(v>=e.deqCost*c||v>=e.deqAvgCost*u)break}else if(g>=e.deqNoDrawCost*i)break;var m=e.deq(t,p,h);if(!(m.length>0))break;for(var b=0;b0&&(e.onDeqd(t,d),!o&&e.shouldRedraw(t,d,p,h)&&a())},s=e.priority||n.noop;r.beforeRender(o,s(t))}}}}},{"../../../util":100}],78:[function(e,t,r){"use strict";t.exports=[{name:"null",impl:e("./null")},{name:"base",impl:e("./base")},{name:"canvas",impl:e("./canvas")}]},{"./base":60,"./canvas":74,"./null":79}],79:[function(e,t,r){"use strict";function n(e){this.options=e,this.notifications=0}var i=function(){};n.prototype={recalculateRenderedStyle:i,notify:function(){this.notifications++},init:i},t.exports=n},{}],80:[function(e,t,r){/*! Weaver licensed under MIT (https://tldrlegal.com/license/mit-license), copyright Max Franz */ +"use strict";var n=e("./is"),i=e("./util"),a=e("./thread"),o=e("./promise"),s=e("./define"),l=function(t){if(!(this instanceof l))return new l(t);this._private={pass:[]};var r=4;if(n.number(t),"undefined"!=typeof navigator&&null!=navigator.hardwareConcurrency)t=navigator.hardwareConcurrency;else try{t=e("os").cpus().length}catch(i){t=r}for(var o=0;t>o;o++)this[o]=new a;this.length=t},u=l.prototype;i.extend(u,{instanceString:function(){return"fabric"},require:function(e,t){for(var r=0;re?-1:e>t?1:0},t.require(e,"_$_$_cmp"),t.spread(function(e){var t=e.sort(_$_$_cmp);resolve(t)}).then(function(t){for(var i=function(n,i,a){i=Math.min(i,r),a=Math.min(a,r);for(var o=n,s=i,l=[],u=o;a>u;u++){var c=t[n],d=t[i];s>n&&(i>=a||e(c,d)<=0)?(l.push(c),n++):(l.push(d),i++)}for(var u=0;ua;a*=2)for(var o=0;r>o;o+=2*a)i(o,o+a,o+2*a);return t})}});var c=function(e){return e=e||{},function(t,r){var n=this._private.pass.shift();return this.random().pass(n)[e.threadFn](t,r)}};i.extend(u,{randomMap:c({threadFn:"map"}),reduce:c({threadFn:"reduce"}),reduceRight:c({threadFn:"reduceRight"})});var d=u;d.promise=d.run,d.terminate=d.halt=d.stop,d.include=d.require,i.extend(u,{on:s.on(),one:s.on({unbindSelfOnTrigger:!0}),off:s.off(),trigger:s.trigger()}),s.eventAliasesOn(u),t.exports=l},{"./define":44,"./is":83,"./promise":86,"./thread":98,"./util":100,os:void 0}],81:[function(e,t,r){/*! +Ported by Xueqiao Xu ; + +PSF LICENSE AGREEMENT FOR PYTHON 2.7.2 + +1. This LICENSE AGREEMENT is between the Python Software Foundation (“PSF”), and the Individual or Organization (“Licensee”) accessing and otherwise using Python 2.7.2 software in source or binary form and its associated documentation. +2. Subject to the terms and conditions of this License Agreement, PSF hereby grants Licensee a nonexclusive, royalty-free, world-wide license to reproduce, analyze, test, perform and/or display publicly, prepare derivative works, distribute, and otherwise use Python 2.7.2 alone or in any derivative version, provided, however, that PSF’s License Agreement and PSF’s notice of copyright, i.e., “Copyright © 2001-2012 Python Software Foundation; All Rights Reserved” are retained in Python 2.7.2 alone or in any derivative version prepared by Licensee. +3. In the event Licensee prepares a derivative work that is based on or incorporates Python 2.7.2 or any part thereof, and wants to make the derivative work available to others as provided herein, then Licensee hereby agrees to include in any such work a brief summary of the changes made to Python 2.7.2. +4. PSF is making Python 2.7.2 available to Licensee on an “AS IS” basis. PSF MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, PSF MAKES NO AND DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF PYTHON 2.7.2 WILL NOT INFRINGE ANY THIRD PARTY RIGHTS. +5. PSF SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF PYTHON 2.7.2 FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS AS A RESULT OF MODIFYING, DISTRIBUTING, OR OTHERWISE USING PYTHON 2.7.2, OR ANY DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. +6. This License Agreement will automatically terminate upon a material breach of its terms and conditions. +7. Nothing in this License Agreement shall be deemed to create any relationship of agency, partnership, or joint venture between PSF and Licensee. This License Agreement does not grant permission to use PSF trademarks or trade name in a trademark sense to endorse or promote products or services of Licensee, or any third party. +8. By copying, installing or otherwise using Python 2.7.2, Licensee agrees to be bound by the terms and conditions of this License Agreement. +*/ +"use strict";var n,i,a,o,s,l,u,c,d,h,p,f,v,g,y;a=Math.floor,h=Math.min,i=function(e,t){return t>e?-1:e>t?1:0},d=function(e,t,r,n,o){var s;if(null==r&&(r=0),null==o&&(o=i),0>r)throw new Error("lo must be non-negative");for(null==n&&(n=e.length);n>r;)s=a((r+n)/2),o(t,e[s])<0?n=s:r=s+1;return[].splice.apply(e,[r,r-r].concat(t)),t},l=function(e,t,r){return null==r&&(r=i),e.push(t),g(e,0,e.length-1,r)},s=function(e,t){var r,n;return null==t&&(t=i),r=e.pop(),e.length?(n=e[0],e[0]=r,y(e,0,t)):n=r,n},c=function(e,t,r){var n;return null==r&&(r=i),n=e[0],e[0]=t,y(e,0,r),n},u=function(e,t,r){var n;return null==r&&(r=i),e.length&&r(e[0],t)<0&&(n=[e[0],t],t=n[0],e[0]=n[1],y(e,0,r)),t},o=function(e,t){var r,n,o,s,l,u;for(null==t&&(t=i),s=function(){u=[];for(var t=0,r=a(e.length/2);r>=0?r>t:t>r;r>=0?t++:t--)u.push(t);return u}.apply(this).reverse(),l=[],n=0,o=s.length;o>n;n++)r=s[n],l.push(y(e,r,t));return l},v=function(e,t,r){var n;return null==r&&(r=i),n=e.indexOf(t),-1!==n?(g(e,0,n,r),y(e,n,r)):void 0},p=function(e,t,r){var n,a,s,l,c;if(null==r&&(r=i),a=e.slice(0,t),!a.length)return a;for(o(a,r),c=e.slice(t),s=0,l=c.length;l>s;s++)n=c[s],u(a,n,r);return a.sort(r).reverse()},f=function(e,t,r){var n,a,l,u,c,p,f,v,g,y;if(null==r&&(r=i),10*t<=e.length){if(u=e.slice(0,t).sort(r),!u.length)return u;for(l=u[u.length-1],v=e.slice(t),c=0,f=v.length;f>c;c++)n=v[c],r(n,l)<0&&(d(u,n,0,null,r),u.pop(),l=u[u.length-1]);return u}for(o(e,r),y=[],a=p=0,g=h(t,e.length);g>=0?g>p:p>g;a=g>=0?++p:--p)y.push(s(e,r));return y},g=function(e,t,r,n){var a,o,s;for(null==n&&(n=i),a=e[r];r>t&&(s=r-1>>1,o=e[s],n(a,o)<0);)e[r]=o,r=s;return e[r]=a},y=function(e,t,r){var n,a,o,s,l;for(null==r&&(r=i),a=e.length,l=t,o=e[t],n=2*t+1;a>n;)s=n+1,a>s&&!(r(e[n],e[s])<0)&&(n=s),e[t]=e[n],t=n,n=2*t+1;return e[t]=o,g(e,l,t,r)},n=function(){function e(e){this.cmp=null!=e?e:i,this.nodes=[]}return e.push=l,e.pop=s,e.replace=c,e.pushpop=u,e.heapify=o,e.updateItem=v,e.nlargest=p,e.nsmallest=f,e.prototype.push=function(e){return l(this.nodes,e,this.cmp)},e.prototype.pop=function(){return s(this.nodes,this.cmp)},e.prototype.peek=function(){return this.nodes[0]},e.prototype.contains=function(e){return-1!==this.nodes.indexOf(e)},e.prototype.replace=function(e){return c(this.nodes,e,this.cmp)},e.prototype.pushpop=function(e){return u(this.nodes,e,this.cmp)},e.prototype.heapify=function(){return o(this.nodes,this.cmp)},e.prototype.updateItem=function(e){return v(this.nodes,e,this.cmp)},e.prototype.clear=function(){return this.nodes=[]},e.prototype.empty=function(){return 0===this.nodes.length},e.prototype.size=function(){return this.nodes.length},e.prototype.clone=function(){var t;return t=new e,t.nodes=this.nodes.slice(0),t},e.prototype.toArray=function(){return this.nodes.slice(0)},e.prototype.insert=e.prototype.push,e.prototype.top=e.prototype.peek,e.prototype.front=e.prototype.peek,e.prototype.has=e.prototype.contains,e.prototype.copy=e.prototype.clone,e}(),t.exports=n},{}],82:[function(e,t,r){"use strict";e("./-preamble");var n=e("./window"),i=e("./is"),a=e("./core"),o=e("./extension"),s=e("./jquery-plugin"),l=e("./stylesheet"),u=e("./thread"),c=e("./fabric"),d=function(e){return void 0===e&&(e={}),i.plainObject(e)?new a(e):i.string(e)?o.apply(o,arguments):void 0};d.version=e("./version.json"),n&&n.jQuery&&s(n.jQuery,d),d.registerJquery=function(e){s(e,d)},d.stylesheet=d.Stylesheet=l,d.thread=d.Thread=u,d.fabric=d.Fabric=c,t.exports=d},{"./-preamble":1,"./core":37,"./extension":46,"./fabric":80,"./is":83,"./jquery-plugin":84,"./stylesheet":97,"./thread":98,"./version.json":106,"./window":107}],83:[function(e,t,r){"use strict";var n=e("./window"),i=n?n.navigator:null,a=n?n.document:null,o="string",s=typeof{},l="function",u=typeof HTMLElement,c=function(e){return e&&e.instanceString&&d.fn(e.instanceString)?e.instanceString():null},d={defined:function(e){return null!=e},string:function(e){return null!=e&&typeof e==o},fn:function(e){return null!=e&&typeof e===l},array:function(e){return Array.isArray?Array.isArray(e):null!=e&&e instanceof Array},plainObject:function(e){return null!=e&&typeof e===s&&!d.array(e)&&e.constructor===Object},object:function(e){return null!=e&&typeof e===s},number:function(e){return null!=e&&"number"==typeof e&&!isNaN(e)},integer:function(e){return d.number(e)&&Math.floor(e)===e},bool:function(e){return null!=e&&typeof e==typeof!0},htmlElement:function(e){return"undefined"===u?void 0:null!=e&&e instanceof HTMLElement},elementOrCollection:function(e){return d.element(e)||d.collection(e)},element:function(e){return"collection"===c(e)&&e._private.single},collection:function(e){return"collection"===c(e)&&!e._private.single},core:function(e){return"core"===c(e)},style:function(e){return"style"===c(e)},stylesheet:function(e){return"stylesheet"===c(e)},event:function(e){return"event"===c(e)},thread:function(e){return"thread"===c(e)},fabric:function(e){return"fabric"===c(e)},emptyString:function(e){return void 0===e||null===e?!0:!(""!==e&&!e.match(/^\s+$/))},nonemptyString:function(e){return!(!e||!d.string(e)||""===e||e.match(/^\s+$/))},domElement:function(e){return"undefined"==typeof HTMLElement?!1:e instanceof HTMLElement},boundingBox:function(e){return d.plainObject(e)&&d.number(e.x1)&&d.number(e.x2)&&d.number(e.y1)&&d.number(e.y2)},promise:function(e){return d.object(e)&&d.fn(e.then)},touch:function(){return n&&("ontouchstart"in n||n.DocumentTouch&&a instanceof DocumentTouch)},gecko:function(){return n&&("undefined"!=typeof InstallTrigger||"MozAppearance"in a.documentElement.style)},webkit:function(){return n&&("undefined"!=typeof webkitURL||"WebkitAppearance"in a.documentElement.style)},chromium:function(){return n&&"undefined"!=typeof chrome},khtml:function(){return i&&i.vendor.match(/kde/i)},khtmlEtc:function(){return d.khtml()||d.webkit()||d.chromium()},ms:function(){return i&&i.userAgent.match(/msie|trident|edge/i)},windows:function(){return i&&i.appVersion.match(/Win/i)},mac:function(){return i&&i.appVersion.match(/Mac/i)},linux:function(){return i&&i.appVersion.match(/Linux/i)},unix:function(){return i&&i.appVersion.match(/X11/i)}};t.exports=d},{"./window":107}],84:[function(e,t,r){"use strict";var n=e("./is"),i=function(e){var t=e[0]._cyreg=e[0]._cyreg||{};return t},a=function(e,t){e&&(e.fn.cytoscape||(e.fn.cytoscape=function(r){var a=e(this);if("get"===r)return i(a).cy;if(n.fn(r)){var o=r,s=i(a).cy;if(s&&s.isReady())s.trigger("ready",[],o);else{var l=i(a),u=l.readies=l.readies||[];u.push(o)}}else if(n.plainObject(r))return a.each(function(){var n=e.extend({},r,{container:e(this)[0]});t(n)})},e.cytoscape=t,null==e.fn.cy&&null==e.cy&&(e.fn.cy=e.fn.cytoscape,e.cy=e.cytoscape)))};t.exports=a},{"./is":83}],85:[function(e,t,r){"use strict";var n={};n.arePositionsSame=function(e,t){return e.x===t.x&&e.y===t.y},n.copyPosition=function(e){return{x:e.x,y:e.y}},n.array2point=function(e){return{x:e[0],y:e[1]}},n.deg2rad=function(e){return Math.PI*e/180},n.log2=Math.log2||function(e){return Math.log(e)/Math.log(2)},n.signum=function(e){return e>0?1:0>e?-1:0},n.dist=function(e,t){return Math.sqrt(n.sqdist(e,t))},n.sqdist=function(e,t){var r=t.x-e.x,n=t.y-e.y;return r*r+n*n},n.qbezierAt=function(e,t,r,n){return(1-n)*(1-n)*e+2*(1-n)*n*t+n*n*r},n.qbezierPtAt=function(e,t,r,i){return{x:n.qbezierAt(e.x,t.x,r.x,i),y:n.qbezierAt(e.y,t.y,r.y,i)}},n.lineAt=function(e,t,r,i){var a={x:t.x-e.x,y:t.y-e.y},o=n.dist(e,t),s={x:a.x/o,y:a.y/o};r=null==r?0:r;var i=null!=i?i:r*o;return{x:e.x+s.x*i,y:e.y+s.y*i}},n.lineAtDist=function(e,t,r){return n.lineAt(e,t,void 0,r)},n.triangleAngle=function(e,t,r){var i=n.dist(t,r),a=n.dist(e,r),o=n.dist(e,t);return Math.acos((i*i+a*a-o*o)/(2*i*a))},n.bound=function(e,t,r){return Math.max(e,Math.min(r,t))},n.makeBoundingBox=function(e){if(null==e)return{x1:1/0,y1:1/0,x2:-(1/0),y2:-(1/0),w:0,h:0};if(null!=e.x1&&null!=e.y1){if(null!=e.x2&&null!=e.y2&&e.x2>=e.x1&&e.y2>=e.y1)return{x1:e.x1,y1:e.y1,x2:e.x2,y2:e.y2,w:e.x2-e.x1,h:e.y2-e.y1};if(null!=e.w&&null!=e.h&&e.w>=0&&e.h>=0)return{x1:e.x1,y1:e.y1,x2:e.x1+e.w,y2:e.y1+e.h,w:e.w,h:e.h}}},n.updateBoundingBox=function(e,t){e.x1=Math.min(e.x1,t.x1),e.x2=Math.max(e.x2,t.x2),e.w=e.x2-e.x1,e.y1=Math.min(e.y1,t.y1),e.y2=Math.max(e.y2,t.y2),e.h=e.y2-e.y1},n.expandBoundingBox=function(e,t){return e.x1-=t,e.x2+=t,e.y1-=t,e.y2+=t,e.w=e.x2-e.x1,e.h=e.y2-e.y1,e},n.boundingBoxesIntersect=function(e,t){return e.x1>t.x2?!1:t.x1>e.x2?!1:e.x2t.y2?!1:!(t.y1>e.y2)},n.inBoundingBox=function(e,t,r){return e.x1<=t&&t<=e.x2&&e.y1<=r&&r<=e.y2},n.pointInBoundingBox=function(e,t){return this.inBoundingBox(e,t.x,t.y)},n.boundingBoxInBoundingBox=function(e,t){return n.inBoundingBox(e,t.x1,t.y1)&&n.inBoundingBox(e,t.x2,t.y2)},n.roundRectangleIntersectLine=function(e,t,r,n,i,a,o){var s,l=this.getRoundRectangleRadius(i,a),u=i/2,c=a/2,d=r-u+l-o,h=n-c-o,p=r+u-l+o,f=h;if(s=this.finiteLinesIntersect(e,t,r,n,d,h,p,f,!1),s.length>0)return s;var v=r+u+o,g=n-c+l-o,y=v,m=n+c-l+o;if(s=this.finiteLinesIntersect(e,t,r,n,v,g,y,m,!1),s.length>0)return s;var b=r-u+l-o,x=n+c+o,w=r+u-l+o,E=x;if(s=this.finiteLinesIntersect(e,t,r,n,b,x,w,E,!1),s.length>0)return s;var _=r-u-o,P=n-c+l-o,S=_,k=n+c-l+o;if(s=this.finiteLinesIntersect(e,t,r,n,_,P,S,k,!1),s.length>0)return s;var T,D=r-u+l,C=n-c+l;if(T=this.intersectLineCircle(e,t,r,n,D,C,l+o),T.length>0&&T[0]<=D&&T[1]<=C)return[T[0],T[1]];var M=r+u-l,N=n-c+l;if(T=this.intersectLineCircle(e,t,r,n,M,N,l+o),T.length>0&&T[0]>=M&&T[1]<=N)return[T[0],T[1]];var B=r+u-l,z=n+c-l;if(T=this.intersectLineCircle(e,t,r,n,B,z,l+o),T.length>0&&T[0]>=B&&T[1]>=z)return[T[0],T[1]];var I=r-u+l,L=n+c-l;return T=this.intersectLineCircle(e,t,r,n,I,L,l+o),T.length>0&&T[0]<=I&&T[1]>=L?[T[0],T[1]]:[]},n.inLineVicinity=function(e,t,r,n,i,a,o){var s=o,l=Math.min(r,i),u=Math.max(r,i),c=Math.min(n,a),d=Math.max(n,a);return e>=l-s&&u+s>=e&&t>=c-s&&d+s>=t},n.inBezierVicinity=function(e,t,r,n,i,a,o,s,l){var u={x1:Math.min(r,o,i)-l,x2:Math.max(r,o,i)+l,y1:Math.min(n,s,a)-l,y2:Math.max(n,s,a)+l};return!(eu.x2||tu.y2)},n.solveCubic=function(e,t,r,n,i){t/=e,r/=e,n/=e;var a,o,s,l,u,c,d,h;return o=(3*r-t*t)/9,s=-(27*n)+t*(9*r-2*(t*t)),s/=54,a=o*o*o+s*s,i[1]=0,d=t/3,a>0?(u=s+Math.sqrt(a),u=0>u?-Math.pow(-u,1/3):Math.pow(u,1/3),c=s-Math.sqrt(a),c=0>c?-Math.pow(-c,1/3):Math.pow(c,1/3),i[0]=-d+u+c,d+=(u+c)/2,i[4]=i[2]=-d,d=Math.sqrt(3)*(-c+u)/2,i[3]=d,void(i[5]=-d)):(i[5]=i[3]=0,0===a?(h=0>s?-Math.pow(-s,1/3):Math.pow(s,1/3),i[0]=-d+2*h,void(i[4]=i[2]=-(h+d))):(o=-o,l=o*o*o,l=Math.acos(s/Math.sqrt(l)),h=2*Math.sqrt(o),i[0]=-d+h*Math.cos(l/3),i[2]=-d+h*Math.cos((l+2*Math.PI)/3),void(i[4]=-d+h*Math.cos((l+4*Math.PI)/3))))},n.sqdistToQuadraticBezier=function(e,t,r,n,i,a,o,s){var l=1*r*r-4*r*i+2*r*o+4*i*i-4*i*o+o*o+n*n-4*n*a+2*n*s+4*a*a-4*a*s+s*s,u=9*r*i-3*r*r-3*r*o-6*i*i+3*i*o+9*n*a-3*n*n-3*n*s-6*a*a+3*a*s,c=3*r*r-6*r*i+r*o-r*e+2*i*i+2*i*e-o*e+3*n*n-6*n*a+n*s-n*t+2*a*a+2*a*t-s*t,d=1*r*i-r*r+r*e-i*e+n*a-n*n+n*t-a*t,h=[];this.solveCubic(l,u,c,d,h);for(var p=1e-7,f=[],v=0;6>v;v+=2)Math.abs(h[v+1])=0&&h[v]<=1&&f.push(h[v]);f.push(1),f.push(0);for(var g,y,m,b,x=-1,w=0;w=0?x>b&&(x=b,g=f[w]):(x=b,g=f[w]);return x},n.sqdistToFiniteLine=function(e,t,r,n,i,a){var o=[e-r,t-n],s=[i-r,a-n],l=s[0]*s[0]+s[1]*s[1],u=o[0]*o[0]+o[1]*o[1],c=o[0]*s[0]+o[1]*s[1],d=c*c/l;return 0>c?u:d>l?(e-i)*(e-i)+(t-a)*(t-a):u-d},n.pointInsidePolygonPoints=function(e,t,r){for(var n,i,a,o,s,l=0,u=0,c=0;c=e&&e>=a||e>=n&&a>=e))continue;s=(e-n)/(a-n)*(o-i)+i,s>t&&l++,t>s&&u++}return l%2!==0},n.pointInsidePolygon=function(e,t,r,i,a,o,s,l,u){var c,d=new Array(r.length);null!=l[0]?(c=Math.atan(l[1]/l[0]),l[0]<0?c+=Math.PI/2:c=-c-Math.PI/2):c=l;for(var h=Math.cos(-c),p=Math.sin(-c),f=0;f0){var g=this.expandPolygon(d,-u);v=this.joinLines(g)}else v=d;return n.pointInsidePolygonPoints(e,t,v)},n.joinLines=function(e){for(var t,r,n,i,a,o,s,l,u=new Array(e.length/2),c=0;cu)return[];var c=u/l;return[(r-e)*c+e,(n-t)*c+t]},n.intersectLineCircle=function(e,t,r,n,i,a,o){var s=[r-e,n-t],l=[i,a],u=[e-i,t-a],c=s[0]*s[0]+s[1]*s[1],d=2*(u[0]*s[0]+u[1]*s[1]),l=u[0]*u[0]+u[1]*u[1]-o*o,h=d*d-4*c*l;if(0>h)return[];var p=(-d+Math.sqrt(h))/(2*c),f=(-d-Math.sqrt(h))/(2*c),v=Math.min(p,f),g=Math.max(p,f),y=[];if(v>=0&&1>=v&&y.push(v),g>=0&&1>=g&&y.push(g),0===y.length)return[];var m=y[0]*s[0]+e,b=y[0]*s[1]+t;if(y.length>1){if(y[0]==y[1])return[m,b];var x=y[1]*s[0]+e,w=y[1]*s[1]+t;return[m,b,x,w]}return[m,b]},n.findCircleNearPoint=function(e,t,r,n,i){var a=n-e,o=i-t,s=Math.sqrt(a*a+o*o),l=a/s,u=o/s;return[e+l*r,t+u*r]},n.findMaxSqDistanceToOrigin=function(e){for(var t,r=1e-6,n=0;nr&&(r=t);return r},n.midOfThree=function(e,t,r){return e>=t&&r>=e||e>=r&&t>=e?e:t>=e&&r>=t||t>=r&&e>=t?t:r},n.finiteLinesIntersect=function(e,t,r,n,i,a,o,s,l){var u=e-i,c=r-e,d=o-i,h=t-a,p=n-t,f=s-a,v=d*h-f*u,g=c*h-p*u,y=f*c-d*p;if(0!==y){var m=v/y,b=g/y,x=.001,w=0-x,E=1+x;return m>=w&&E>=m&&b>=w&&E>=b?[e+m*c,t+m*p]:l?[e+m*c,t+m*p]:[]}return 0===v||0===g?this.midOfThree(e,r,o)===o?[o,s]:this.midOfThree(e,r,i)===i?[i,a]:this.midOfThree(i,o,r)===r?[r,n]:[]:[]},n.polygonIntersectLine=function(e,t,r,i,a,o,s,l){for(var u,c=[],d=new Array(r.length),h=0;h0){var f=n.expandPolygon(d,-l);p=n.joinLines(f)}else p=d;for(var v,g,y,m,h=0;ha&&(a=1e-5),[t[0]+a*n[0],t[1]+a*n[1]]},n.generateUnitNgonPointsFitToSquare=function(e,t){var r=n.generateUnitNgonPoints(e,t);return r=n.fitPolygonToSquare(r)},n.fitPolygonToSquare=function(e){for(var t,r,n=e.length/2,i=1/0,a=1/0,o=-(1/0),s=-(1/0),l=0;n>l;l++)t=e[2*l],r=e[2*l+1],i=Math.min(i,t),o=Math.max(o,t),a=Math.min(a,r),s=Math.max(s,r);for(var u=2/(o-i),c=2/(s-a),l=0;n>l;l++)t=e[2*l]=e[2*l]*u,r=e[2*l+1]=e[2*l+1]*c,i=Math.min(i,t),o=Math.max(o,t),a=Math.min(a,r),s=Math.max(s,r);if(-1>a)for(var l=0;n>l;l++)r=e[2*l+1]=e[2*l+1]+(-1-a);return e},n.generateUnitNgonPoints=function(e,t){var r=1/e*2*Math.PI,n=e%2===0?Math.PI/2+r/2:Math.PI/2;n+=t;for(var i,a,o,s=new Array(2*e),l=0;e>l;l++)i=l*r+n,a=s[2*l]=Math.cos(i),o=s[2*l+1]=Math.sin(-i);return s},n.getRoundRectangleRadius=function(e,t){return Math.min(e/4,t/4,8)},t.exports=n},{}],86:[function(e,t,r){/*! +Embeddable Minimum Strictly-Compliant Promises/A+ 1.1.1 Thenable +Copyright (c) 2013-2014 Ralf S. Engelschall (http://engelschall.com) +Licensed under The MIT License (http://opensource.org/licenses/MIT) +*/ +"use strict";var n=0,i=1,a=2,o=function(e){return this instanceof o?(this.id="Thenable/1.0.7",this.state=n,this.fulfillValue=void 0,this.rejectReason=void 0,this.onFulfilled=[],this.onRejected=[],this.proxy={then:this.then.bind(this)},void("function"==typeof e&&e.call(this,this.fulfill.bind(this),this.reject.bind(this)))):new o(e)};o.prototype={fulfill:function(e){return s(this,i,"fulfillValue",e)},reject:function(e){return s(this,a,"rejectReason",e)},then:function(e,t){var r=this,n=new o;return r.onFulfilled.push(c(e,n,"fulfill")),r.onRejected.push(c(t,n,"reject")),l(r),n.proxy}};var s=function(e,t,r,i){return e.state===n&&(e.state=t,e[r]=i,l(e)),e},l=function(e){e.state===i?u(e,"onFulfilled",e.fulfillValue):e.state===a&&u(e,"onRejected",e.rejectReason)},u=function(e,t,r){if(0!==e[t].length){var n=e[t];e[t]=[];var i=function(){for(var e=0;e\\?\\@\\[\\]\\^\\`\\{\\|\\}\\~]",comparatorOp:"=|\\!=|>|>=|<|<=|\\$=|\\^=|\\*=",boolOp:"\\?|\\!|\\^",string:'"(?:\\\\"|[^"])+"|'+"'(?:\\\\'|[^'])+'",number:i.regex.number,meta:"degree|indegree|outdegree",separator:"\\s*,\\s*",descendant:"\\s+",child:"\\s+>\\s+",subject:"\\$"};l.variable="(?:[\\w-]|(?:\\\\"+l.metaChar+"))+",l.value=l.string+"|"+l.number,l.className=l.variable,l.id=l.variable;for(var u=function(e){return e.replace(new RegExp("\\\\("+l.metaChar+")","g"),function(e,t,r,n){return t})},c=l.comparatorOp.split("|"),d=0;d=0||"="!==h&&(l.comparatorOp+="|\\!"+h)}var p=[{name:"group",query:!0,regex:"(node|edge|\\*)",populate:function(e){this.group="*"===e?e:e+"s"}},{name:"state",query:!0,regex:"(:selected|:unselected|:locked|:unlocked|:visible|:hidden|:transparent|:grabbed|:free|:removed|:inside|:grabbable|:ungrabbable|:animated|:unanimated|:selectable|:unselectable|:orphan|:nonorphan|:parent|:child|:loop|:simple|:active|:inactive|:touch|:backgrounding|:nonbackgrounding)",populate:function(e){this.colonSelectors.push(e)}},{name:"id",query:!0,regex:"\\#("+l.id+")",populate:function(e){this.ids.push(u(e))}},{name:"className",query:!0,regex:"\\.("+l.className+")",populate:function(e){this.classes.push(u(e))}},{name:"dataExists",query:!0,regex:"\\[\\s*("+l.variable+")\\s*\\]",populate:function(e){this.data.push({field:u(e)})}},{name:"dataCompare",query:!0,regex:"\\[\\s*("+l.variable+")\\s*("+l.comparatorOp+")\\s*("+l.value+")\\s*\\]",populate:function(e,t,r){var n=null!=new RegExp("^"+l.string+"$").exec(r);r=n?r.substring(1,r.length-1):parseFloat(r),this.data.push({field:u(e),operator:t,value:r})}},{name:"dataBool",query:!0,regex:"\\[\\s*("+l.boolOp+")\\s*("+l.variable+")\\s*\\]",populate:function(e,t){this.data.push({field:u(t),operator:e})}},{name:"metaCompare",query:!0,regex:"\\[\\[\\s*("+l.meta+")\\s*("+l.comparatorOp+")\\s*("+l.number+")\\s*\\]\\]",populate:function(e,t,r){this.meta.push({field:u(e),operator:t,value:parseFloat(r)})}},{name:"nextQuery",separator:!0,regex:l.separator,populate:function(){t[++d]=r(),s=null}},{name:"child",separator:!0,regex:l.child,populate:function(){var e=r();e.parent=this,e.subject=s,t[d]=e}},{name:"descendant",separator:!0,regex:l.descendant,populate:function(){var e=r();e.ancestor=this,e.subject=s,t[d]=e}},{name:"subject",modifier:!0,regex:l.subject,populate:function(){return null!=s&&this.subject!=this?(i.error("Redefinition of subject in selector `"+e+"`"),!1):(s=this,void(this.subject=this))}}];t._private.selectorText=e;var f=e,d=0,v=function(e){for(var t,r,i,a=0;a=0&&(d=d.toLowerCase(),h=h.toLowerCase(),s=s.replace("@",""),p=!0);var f=!1;s.indexOf("!")>=0&&(s=s.replace("!",""),f=!0),p&&(l=h.toLowerCase(),c=d.toLowerCase());var v=!1;switch(s){case"*=":a=d.indexOf(h)>=0;break;case"$=":a=d.indexOf(h,d.length-h.length)>=0;break;case"^=":a=0===d.indexOf(h);break;case"=":a=c===l;break;case">":v=!0,a=c>l;break;case">=":v=!0,a=c>=l;break;case"<":v=!0,a=l>c;break;case"<=":v=!0,a=l>=c;break;default:a=!1}!f||null==c&&v||(a=!a)}else if(null!=s)switch(s){case"?":a=t.fieldTruthy(u);break;case"!":a=!t.fieldTruthy(u);break;case"^":a=t.fieldUndefined(u)}else a=!t.fieldUndefined(u);if(!a){r=!1;break}}return r},v=f({name:"data",fieldValue:function(e){return r.data[e]},fieldUndefined:function(e){return void 0===r.data[e]},fieldTruthy:function(e){return!!r.data[e]}});if(!v)return!1;var g=f({name:"meta",fieldValue:function(e){return t[e]()},fieldUndefined:function(e){return null==t[e]()},fieldTruthy:function(e){return!!t[e]()}});if(!g)return!1;if(null!=e.collection){var y=e.collection.hasElementWithId(t.id());if(!y)return!1}if(null!=e.filter&&0===t.collection().filter(e.filter).size())return!1;var m=function(e,t){if(null!=e){var r=!1;if(!i.hasCompoundNodes())return!1;t=t();for(var n=0;n "+n),null!=e.ancestor&&(n=a(e.ancestor)+" "+n),null!=e.child&&(n+=" > "+a(e.child)),null!=e.descendant&&(n+=" "+a(e.descendant)),n},o=0;o1&&o0;if(h||p){var f;h&&p?f=u.properties:h?f=u.properties:p&&(f=u.mappedProperties);for(var v=0;v0){n=!0;break}}t.hasPie=n;var o=e.pstyle("text-transform").strValue,s=e.pstyle("label").strValue,l=e.pstyle("source-label").strValue,u=e.pstyle("target-label").strValue,c=e.pstyle("font-style").strValue,a=e.pstyle("font-size").pfValue+"px",d=e.pstyle("font-family").strValue,h=e.pstyle("font-weight").strValue,p=e.pstyle("text-valign").strValue,f=e.pstyle("text-valign").strValue,v=e.pstyle("text-outline-width").pfValue,g=e.pstyle("text-wrap").strValue,y=e.pstyle("text-max-width").pfValue,m=c+"$"+a+"$"+d+"$"+h+"$"+o+"$"+p+"$"+f+"$"+v+"$"+g+"$"+y;t.labelStyleKey=m,t.sourceLabelKey=m+"$"+l,t.targetLabelKey=m+"$"+u,t.labelKey=m+"$"+s,t.fontKey=c+"$"+h+"$"+a+"$"+d,t.styleKey=Date.now()}},a.applyParsedProperty=function(e,t){var r,a,o=this,s=t,l=e._private.style,u=o.types,c=o.properties[s.name].type,d=s.bypass,h=l[s.name],p=h&&h.bypass,f=e._private;if("curve-style"===t.name&&"haystack"===t.value&&e.isEdge()&&(e.isLoop()||e.source().isParent()||e.target().isParent())&&(s=t=this.parse(t.name,"bezier",d)),s["delete"])return l[s.name]=void 0,!0;if(s.deleteBypassed)return h?h.bypass?(h.bypassed=void 0,!0):!1:!0;if(s.deleteBypass)return h?h.bypass?(l[s.name]=h.bypassed,!0):!1:!0;var v=function(){n.error("Do not assign mappings to elements without corresponding data (e.g. ele `"+e.id()+"` for property `"+s.name+"` with data field `"+s.field+"`); try a `["+s.field+"]` selector to limit scope to elements with `"+s.field+"` defined")};switch(s.mapped){case u.mapData:case u.mapLayoutData:case u.mapScratch:var r,g=s.mapped===u.mapLayoutData,y=s.mapped===u.mapScratch,m=s.field.split(".");r=y||g?f.scratch:f.data;for(var b=0;bw?w=0:w>1&&(w=1),c.color){var E=s.valueMin[0],_=s.valueMax[0],P=s.valueMin[1],S=s.valueMax[1],k=s.valueMin[2],T=s.valueMax[2],D=null==s.valueMin[3]?1:s.valueMin[3],C=null==s.valueMax[3]?1:s.valueMax[3],M=[Math.round(E+(_-E)*w),Math.round(P+(S-P)*w),Math.round(k+(T-k)*w),Math.round(D+(C-D)*w)];a={bypass:s.bypass,name:s.name,value:M,strValue:"rgb("+M[0]+", "+M[1]+", "+M[2]+")"}}else{if(!c.number)return!1;var N=s.valueMin+(s.valueMax-s.valueMin)*w;a=this.parse(s.name,N,s.bypass,!0)}a||(a=this.parse(s.name,h.strValue,s.bypass,!0)),a||v(),a.mapping=s,s=a;break;case u.data:case u.layoutData:case u.scratch:var r,g=s.mapped===u.layoutData,y=s.mapped===u.scratch,m=s.field.split(".");if(r=y||g?f.scratch:f.data)for(var b=0;b0&&s>0){for(var u={},c=!1,d=0;d0&&e.delay(l),e.animate({css:u},{duration:s,easing:e.pstyle("transition-timing-function").value,queue:!1,complete:function(){r||n.removeBypasses(e,o),a.transitioning=!1}})}else a.transitioning&&(e.stop(),this.removeBypasses(e,o),a.transitioning=!1)},t.exports=a},{"../is":83,"../util":100}],89:[function(e,t,r){"use strict";var n=e("../is"),i=e("../util"),a={};a.applyBypass=function(e,t,r,a){var o=this,s=[],l=!0;if("*"===t||"**"===t){if(void 0!==r)for(var u=0;uh.max)return null;var B={name:e,value:t,strValue:""+t+(D?D:""),units:D,bypass:r};return h.unitless||"px"!==D&&"em"!==D?B.pfValue=t:B.pfValue="px"!==D&&D?this.getEmSizeInPixels()*t:t,"ms"!==D&&"s"!==D||(B.pfValue="ms"===D?t:1e3*t),"deg"!==D&&"rad"!==D||(B.pfValue="rad"===D?t:a.deg2rad(t)),B}if(h.propList){var z=[],I=""+t;if("none"===I);else{for(var L=I.split(","),O=0;O node").css({shape:"rectangle",padding:10,"background-color":"#eee","border-color":"#ccc","border-width":1}).selector("edge").css({width:3,"curve-style":"haystack"}).selector(":selected").css({"background-color":"#0169D9","line-color":"#0169D9","source-arrow-color":"#0169D9","target-arrow-color":"#0169D9","mid-source-arrow-color":"#0169D9","mid-target-arrow-color":"#0169D9"}).selector("node:parent:selected").css({"background-color":"#CCE1F9","border-color":"#aec8e5"}).selector(":active").css({"overlay-color":"black","overlay-padding":10,"overlay-opacity":.25}).selector("core").css({"selection-box-color":"#ddd","selection-box-opacity":.65,"selection-box-border-color":"#aaa","selection-box-border-width":1,"active-bg-color":"black","active-bg-opacity":.15,"active-bg-size":30,"outside-texture-bg-color":"#000","outside-texture-bg-opacity":.125}),this.defaultLength=this.length},t.exports=i},{"../util":100}],96:[function(e,t,r){"use strict";var n=e("../util"),i=e("../selector"),a={};a.applyFromString=function(e){function t(){c=c.length>a.length?c.substr(a.length):""}function r(){o=o.length>s.length?o.substr(s.length):""}var a,o,s,l=this,u=this,c=""+e;for(c=c.replace(/[\/][*](\s|.)+?[*][\/]/g,"");;){var d=c.match(/^\s*$/);if(d)break;var h=c.match(/^\s*((?:.|\s)+?)\s*\{((?:.|\s)+?)\}/);if(!h){n.error("Halting stylesheet parsing: String stylesheet contains more to parse but no selector and block found in: "+c);break}a=h[0];var p=h[1];if("core"!==p){var f=new i(p);if(f._private.invalid){n.error("Skipping parsing of block: Invalid selector found in string stylesheet: "+p),t();continue}}var v=h[2],g=!1;o=v;for(var y=[];;){var d=o.match(/^\s*$/);if(d)break;var m=o.match(/^\s*(.+?)\s*:\s*(.+?)\s*;/);if(!m){n.error("Skipping parsing of block: Invalid formatting of style property and value definitions found in:"+v),g=!0;break}s=m[0];var b=m[1],x=m[2],w=l.properties[b];if(w){var E=u.parse(b,x);E?(y.push({name:b,val:x}),r()):(n.error("Skipping property: Invalid property definition in: "+s),r())}else n.error("Skipping property: Invalid property name in: "+s),r()}if(g){t();break}u.selector(p);for(var _=0;_1?", "+JSON.stringify(r):"")+" );"," "," resolve = origResolve;"," resolve( res.length > 0 ? res : ret );","}"].join("\n"))}};util.extend(thdfn,{reduce:defineFnal({name:"reduce"}),reduceRight:defineFnal({name:"reduceRight"}),map:defineFnal({name:"map"})});var fn=thdfn;fn.promise=fn.run,fn.terminate=fn.halt=fn.stop,fn.include=fn.require,util.extend(thdfn,{on:define.on(),one:define.on({unbindSelfOnTrigger:!0}),off:define.off(),trigger:define.trigger()}),define.eventAliasesOn(thdfn),module.exports=Thread},{"./define":44,"./event":45,"./is":83,"./promise":86,"./util":100,"./window":107,child_process:void 0,path:void 0}],99:[function(e,t,r){"use strict";var n=e("../is");t.exports={hex2tuple:function(e){if((4===e.length||7===e.length)&&"#"===e[0]){var t,r,n,i=4===e.length,a=16;return i?(t=parseInt(e[1]+e[1],a),r=parseInt(e[2]+e[2],a),n=parseInt(e[3]+e[3],a)):(t=parseInt(e[1]+e[2],a),r=parseInt(e[3]+e[4],a),n=parseInt(e[5]+e[6],a)),[t,r,n]}},hsl2tuple:function(e){function t(e,t,r){return 0>r&&(r+=1),r>1&&(r-=1),1/6>r?e+6*(t-e)*r:.5>r?t:2/3>r?e+(t-e)*(2/3-r)*6:e}var r,n,i,a,o,s,l,u,c=new RegExp("^"+this.regex.hsla+"$").exec(e);if(c){if(n=parseInt(c[1]),0>n?n=(360- -1*n%360)%360:n>360&&(n%=360),n/=360,i=parseFloat(c[2]),0>i||i>100)return;if(i/=100,a=parseFloat(c[3]),0>a||a>100)return;if(a/=100,o=c[4],void 0!==o&&(o=parseFloat(o),0>o||o>1))return;if(0===i)s=l=u=Math.round(255*a);else{var d=.5>a?a*(1+i):a+i-a*i,h=2*a-d;s=Math.round(255*t(h,d,n+1/3)),l=Math.round(255*t(h,d,n)),u=Math.round(255*t(h,d,n-1/3))}r=[s,l,u,o]}return r},rgb2tuple:function(e){var t,r=new RegExp("^"+this.regex.rgba+"$").exec(e);if(r){t=[];for(var n=[],i=1;3>=i;i++){var a=r[i];if("%"===a[a.length-1]&&(n[i]=!0),a=parseFloat(a),n[i]&&(a=a/100*255),0>a||a>255)return;t.push(Math.floor(a))}var o=n[1]||n[2]||n[3],s=n[1]&&n[2]&&n[3];if(o&&!s)return;var l=r[4];if(void 0!==l){if(l=parseFloat(l),0>l||l>1)return;t.push(l)}}return t},colorname2tuple:function(e){return this.colors[e.toLowerCase()]},color2tuple:function(e){return(n.array(e)?e:null)||this.colorname2tuple(e)||this.hex2tuple(e)||this.rgb2tuple(e)||this.hsl2tuple(e)},colors:{transparent:[0,0,0,0],aliceblue:[240,248,255],antiquewhite:[250,235,215],aqua:[0,255,255],aquamarine:[127,255,212],azure:[240,255,255],beige:[245,245,220],bisque:[255,228,196],black:[0,0,0],blanchedalmond:[255,235,205],blue:[0,0,255],blueviolet:[138,43,226],brown:[165,42,42],burlywood:[222,184,135],cadetblue:[95,158,160],chartreuse:[127,255,0],chocolate:[210,105,30],coral:[255,127,80],cornflowerblue:[100,149,237],cornsilk:[255,248,220],crimson:[220,20,60],cyan:[0,255,255],darkblue:[0,0,139],darkcyan:[0,139,139],darkgoldenrod:[184,134,11],darkgray:[169,169,169],darkgreen:[0,100,0],darkgrey:[169,169,169],darkkhaki:[189,183,107],darkmagenta:[139,0,139],darkolivegreen:[85,107,47],darkorange:[255,140,0],darkorchid:[153,50,204],darkred:[139,0,0],darksalmon:[233,150,122],darkseagreen:[143,188,143],darkslateblue:[72,61,139],darkslategray:[47,79,79],darkslategrey:[47,79,79],darkturquoise:[0,206,209],darkviolet:[148,0,211],deeppink:[255,20,147],deepskyblue:[0,191,255],dimgray:[105,105,105],dimgrey:[105,105,105],dodgerblue:[30,144,255],firebrick:[178,34,34],floralwhite:[255,250,240],forestgreen:[34,139,34],fuchsia:[255,0,255],gainsboro:[220,220,220],ghostwhite:[248,248,255],gold:[255,215,0],goldenrod:[218,165,32],gray:[128,128,128],grey:[128,128,128],green:[0,128,0],greenyellow:[173,255,47],honeydew:[240,255,240],hotpink:[255,105,180],indianred:[205,92,92],indigo:[75,0,130],ivory:[255,255,240],khaki:[240,230,140],lavender:[230,230,250],lavenderblush:[255,240,245],lawngreen:[124,252,0],lemonchiffon:[255,250,205],lightblue:[173,216,230],lightcoral:[240,128,128],lightcyan:[224,255,255],lightgoldenrodyellow:[250,250,210],lightgray:[211,211,211],lightgreen:[144,238,144],lightgrey:[211,211,211],lightpink:[255,182,193],lightsalmon:[255,160,122],lightseagreen:[32,178,170],lightskyblue:[135,206,250],lightslategray:[119,136,153],lightslategrey:[119,136,153],lightsteelblue:[176,196,222],lightyellow:[255,255,224],lime:[0,255,0],limegreen:[50,205,50],linen:[250,240,230],magenta:[255,0,255],maroon:[128,0,0],mediumaquamarine:[102,205,170],mediumblue:[0,0,205],mediumorchid:[186,85,211],mediumpurple:[147,112,219],mediumseagreen:[60,179,113],mediumslateblue:[123,104,238],mediumspringgreen:[0,250,154],mediumturquoise:[72,209,204],mediumvioletred:[199,21,133],midnightblue:[25,25,112],mintcream:[245,255,250],mistyrose:[255,228,225],moccasin:[255,228,181],navajowhite:[255,222,173],navy:[0,0,128],oldlace:[253,245,230],olive:[128,128,0],olivedrab:[107,142,35],orange:[255,165,0],orangered:[255,69,0],orchid:[218,112,214],palegoldenrod:[238,232,170],palegreen:[152,251,152],paleturquoise:[175,238,238],palevioletred:[219,112,147],papayawhip:[255,239,213],peachpuff:[255,218,185],peru:[205,133,63],pink:[255,192,203],plum:[221,160,221],powderblue:[176,224,230],purple:[128,0,128],red:[255,0,0],rosybrown:[188,143,143],royalblue:[65,105,225],saddlebrown:[139,69,19],salmon:[250,128,114],sandybrown:[244,164,96],seagreen:[46,139,87],seashell:[255,245,238],sienna:[160,82,45],silver:[192,192,192],skyblue:[135,206,235],slateblue:[106,90,205],slategray:[112,128,144],slategrey:[112,128,144],snow:[255,250,250],springgreen:[0,255,127],steelblue:[70,130,180],tan:[210,180,140],teal:[0,128,128],thistle:[216,191,216],tomato:[255,99,71],turquoise:[64,224,208],violet:[238,130,238],wheat:[245,222,179],white:[255,255,255],whitesmoke:[245,245,245],yellow:[255,255,0],yellowgreen:[154,205,50]}}},{"../is":83}],100:[function(e,t,r){"use strict";var n=e("../is"),i=e("../math"),a={trueify:function(){return!0},falsify:function(){return!1},zeroify:function(){return 0},noop:function(){},error:function(e){console.error?(console.error.apply(console,arguments),console.trace&&console.trace()):(console.log.apply(console,arguments),console.trace&&console.trace())},clone:function(e){return this.extend({},e)},copy:function(e){return null==e?e:n.array(e)?e.slice():n.plainObject(e)?this.clone(e):e},uuid:function(e,t){for(t=e="";e++<36;t+=51*e&52?(15^e?8^Math.random()*(20^e?16:4):4).toString(16):"-");return t}};a.makeBoundingBox=i.makeBoundingBox.bind(i),a._staticEmptyObject={},a.staticEmptyObject=function(){return a._staticEmptyObject},a.extend=null!=Object.assign?Object.assign:function(e){for(var t=arguments,r=1;r=0&&(e[n]!==t||(e.splice(n,1),r));n--);},a.clearArray=function(e){e.splice(0,e.length)},a.getPrefixedProperty=function(e,t,r){return r&&(t=this.prependCamel(r,t)),e[t]},a.setPrefixedProperty=function(e,t,r,n){r&&(t=this.prependCamel(r,t)),e[t]=n},[e("./colors"),e("./maps"),{memoize:e("./memoize")},e("./regex"),e("./strings"),e("./timing")].forEach(function(e){a.extend(a,e)}),t.exports=a},{"../is":83,"../math":85,"./colors":99,"./maps":101,"./memoize":102,"./regex":103,"./strings":104,"./timing":105}],101:[function(e,t,r){"use strict";var n=e("../is");t.exports={mapEmpty:function(e){var t=!0;return null!=e?0===Object.keys(e).length:t},pushMap:function(e){var t=this.getMap(e);null==t?this.setMap(this.extend({},e,{value:[e.value]})):t.push(e.value)},setMap:function(e){for(var t,r=e.map,i=e.keys,a=i.length,o=0;a>o;o++){var t=i[o];n.plainObject(t)&&this.error("Tried to set map with object key"),oa;a++){var o=r[a];if(n.plainObject(o)&&this.error("Tried to get map with object key"),t=t[o],null==t)return t}return t},deleteMap:function(e){for(var t=e.map,r=e.keys,i=r.length,a=e.keepChildren,o=0;i>o;o++){var s=r[o];n.plainObject(s)&&this.error("Tried to delete map with object key");var l=o===e.keys.length-1;if(l)if(a)for(var u=Object.keys(t),c=0;c=r){a&&clearTimeout(a);var i=c;a=u=c=void 0,i&&(h=d.now(),o=e.apply(l,n),u||a||(n=l=null))}else u=setTimeout(g,r)},y=function(){u&&clearTimeout(u),a=u=c=void 0,(f||p!==t)&&(h=d.now(),o=e.apply(l,n),u||a||(n=l=null))};return function(){if(n=arguments,s=d.now(),l=this,c=f&&(u||!v),p===!1)var r=v&&!u;else{a||v||(h=s);var i=p-(s-h),m=0>=i;m?(a&&(a=clearTimeout(a)),h=s,o=e.apply(l,n)):a||(a=setTimeout(y,i))}return m&&u?u=clearTimeout(u):u||t===p||(u=setTimeout(g,t)),r&&(m=!0,o=e.apply(l,n)),!m||u||a||(n=l=null),o}}},t.exports=o},{"../is":83,"../window":107}],106:[function(e,t,r){t.exports="2.7.14"},{}],107:[function(e,t,r){t.exports="undefined"==typeof window?null:window},{}]},{},[82])(82)}); \ No newline at end of file diff --git a/storm-core/src/ui/public/js/dagre.min.js b/storm-core/src/ui/public/js/dagre.min.js new file mode 100644 index 00000000000..b7a9bbc4dc6 --- /dev/null +++ b/storm-core/src/ui/public/js/dagre.min.js @@ -0,0 +1,6 @@ +!function(e){if("object"==typeof exports&&"undefined"!=typeof module)module.exports=e();else if("function"==typeof define&&define.amd)define([],e);else{var f;"undefined"!=typeof window?f=window:"undefined"!=typeof global?f=global:"undefined"!=typeof self&&(f=self),f.dagre=e()}}(function(){var define,module,exports;return function e(t,n,r){function s(o,u){if(!n[o]){if(!t[o]){var a=typeof require=="function"&&require;if(!u&&a)return a(o,!0);if(i)return i(o,!0);var f=new Error("Cannot find module '"+o+"'");throw f.code="MODULE_NOT_FOUND",f}var l=n[o]={exports:{}};t[o][0].call(l.exports,function(e){var n=t[o][1][e];return s(n?n:e)},l,l.exports,e,t,n,r)}return n[o].exports}var i=typeof require=="function"&&require;for(var o=0;o0;--i){entry=buckets[i].dequeue();if(entry){results=results.concat(removeNode(g,buckets,zeroIdx,entry,true));break}}}}return results}function removeNode(g,buckets,zeroIdx,entry,collectPredecessors){var results=collectPredecessors?[]:undefined;_.each(g.inEdges(entry.v),function(edge){var weight=g.edge(edge),uEntry=g.node(edge.v);if(collectPredecessors){results.push({v:edge.v,w:edge.w})}uEntry.out-=weight;assignBucket(buckets,zeroIdx,uEntry)});_.each(g.outEdges(entry.v),function(edge){var weight=g.edge(edge),w=edge.w,wEntry=g.node(w);wEntry["in"]-=weight;assignBucket(buckets,zeroIdx,wEntry)});g.removeNode(entry.v);return results}function buildState(g,weightFn){var fasGraph=new Graph,maxIn=0,maxOut=0;_.each(g.nodes(),function(v){fasGraph.setNode(v,{v:v,"in":0,out:0})});_.each(g.edges(),function(e){var prevWeight=fasGraph.edge(e.v,e.w)||0,weight=weightFn(e),edgeWeight=prevWeight+weight;fasGraph.setEdge(e.v,e.w,edgeWeight);maxOut=Math.max(maxOut,fasGraph.node(e.v).out+=weight);maxIn=Math.max(maxIn,fasGraph.node(e.w)["in"]+=weight)});var buckets=_.range(maxOut+maxIn+3).map(function(){return new List});var zeroIdx=maxIn+1;_.each(fasGraph.nodes(),function(v){assignBucket(buckets,zeroIdx,fasGraph.node(v))});return{graph:fasGraph,buckets:buckets,zeroIdx:zeroIdx}}function assignBucket(buckets,zeroIdx,entry){if(!entry.out){buckets[0].enqueue(entry)}else if(!entry["in"]){buckets[buckets.length-1].enqueue(entry)}else{buckets[entry.out-entry["in"]+zeroIdx].enqueue(entry)}}},{"./data/list":5,"./graphlib":7,"./lodash":10}],9:[function(require,module,exports){"use strict";var _=require("./lodash"),acyclic=require("./acyclic"),normalize=require("./normalize"),rank=require("./rank"),normalizeRanks=require("./util").normalizeRanks,parentDummyChains=require("./parent-dummy-chains"),removeEmptyRanks=require("./util").removeEmptyRanks,nestingGraph=require("./nesting-graph"),addBorderSegments=require("./add-border-segments"),coordinateSystem=require("./coordinate-system"),order=require("./order"),position=require("./position"),util=require("./util"),Graph=require("./graphlib").Graph;module.exports=layout;function layout(g,opts){var time=opts&&opts.debugTiming?util.time:util.notime;time("layout",function(){var layoutGraph=time(" buildLayoutGraph",function(){return buildLayoutGraph(g)});time(" runLayout",function(){runLayout(layoutGraph,time)});time(" updateInputGraph",function(){updateInputGraph(g,layoutGraph)})})}function runLayout(g,time){time(" makeSpaceForEdgeLabels",function(){makeSpaceForEdgeLabels(g)});time(" removeSelfEdges",function(){removeSelfEdges(g)});time(" acyclic",function(){acyclic.run(g)});time(" nestingGraph.run",function(){nestingGraph.run(g)});time(" rank",function(){rank(util.asNonCompoundGraph(g))});time(" injectEdgeLabelProxies",function(){injectEdgeLabelProxies(g)});time(" removeEmptyRanks",function(){removeEmptyRanks(g)});time(" nestingGraph.cleanup",function(){nestingGraph.cleanup(g)});time(" normalizeRanks",function(){normalizeRanks(g)});time(" assignRankMinMax",function(){assignRankMinMax(g)});time(" removeEdgeLabelProxies",function(){removeEdgeLabelProxies(g)});time(" normalize.run",function(){normalize.run(g)});time(" parentDummyChains",function(){parentDummyChains(g)});time(" addBorderSegments",function(){addBorderSegments(g)});time(" order",function(){order(g)});time(" insertSelfEdges",function(){insertSelfEdges(g)});time(" adjustCoordinateSystem",function(){coordinateSystem.adjust(g)});time(" position",function(){position(g)});time(" positionSelfEdges",function(){positionSelfEdges(g)});time(" removeBorderNodes",function(){removeBorderNodes(g)});time(" normalize.undo",function(){normalize.undo(g)});time(" fixupEdgeLabelCoords",function(){fixupEdgeLabelCoords(g)});time(" undoCoordinateSystem",function(){coordinateSystem.undo(g)});time(" translateGraph",function(){translateGraph(g)});time(" assignNodeIntersects",function(){assignNodeIntersects(g)});time(" reversePoints",function(){reversePointsForReversedEdges(g)});time(" acyclic.undo",function(){acyclic.undo(g)})}function updateInputGraph(inputGraph,layoutGraph){_.each(inputGraph.nodes(),function(v){var inputLabel=inputGraph.node(v),layoutLabel=layoutGraph.node(v);if(inputLabel){inputLabel.x=layoutLabel.x;inputLabel.y=layoutLabel.y;if(layoutGraph.children(v).length){inputLabel.width=layoutLabel.width;inputLabel.height=layoutLabel.height}}});_.each(inputGraph.edges(),function(e){var inputLabel=inputGraph.edge(e),layoutLabel=layoutGraph.edge(e);inputLabel.points=layoutLabel.points;if(_.has(layoutLabel,"x")){inputLabel.x=layoutLabel.x;inputLabel.y=layoutLabel.y}});inputGraph.graph().width=layoutGraph.graph().width;inputGraph.graph().height=layoutGraph.graph().height}var graphNumAttrs=["nodesep","edgesep","ranksep","marginx","marginy"],graphDefaults={ranksep:50,edgesep:20,nodesep:50,rankdir:"tb"},graphAttrs=["acyclicer","ranker","rankdir","align"],nodeNumAttrs=["width","height"],nodeDefaults={width:0,height:0},edgeNumAttrs=["minlen","weight","width","height","labeloffset"],edgeDefaults={minlen:1,weight:1,width:0,height:0,labeloffset:10,labelpos:"r"},edgeAttrs=["labelpos"];function buildLayoutGraph(inputGraph){var g=new Graph({multigraph:true,compound:true}),graph=canonicalize(inputGraph.graph());g.setGraph(_.merge({},graphDefaults,selectNumberAttrs(graph,graphNumAttrs),_.pick(graph,graphAttrs)));_.each(inputGraph.nodes(),function(v){var node=canonicalize(inputGraph.node(v));g.setNode(v,_.defaults(selectNumberAttrs(node,nodeNumAttrs),nodeDefaults));g.setParent(v,inputGraph.parent(v))});_.each(inputGraph.edges(),function(e){var edge=canonicalize(inputGraph.edge(e));g.setEdge(e,_.merge({},edgeDefaults,selectNumberAttrs(edge,edgeNumAttrs),_.pick(edge,edgeAttrs)))});return g}function makeSpaceForEdgeLabels(g){var graph=g.graph();graph.ranksep/=2;_.each(g.edges(),function(e){var edge=g.edge(e);edge.minlen*=2;if(edge.labelpos.toLowerCase()!=="c"){if(graph.rankdir==="TB"||graph.rankdir==="BT"){edge.width+=edge.labeloffset}else{edge.height+=edge.labeloffset}}})}function injectEdgeLabelProxies(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.width&&edge.height){var v=g.node(e.v),w=g.node(e.w),label={rank:(w.rank-v.rank)/2+v.rank,e:e};util.addDummyNode(g,"edge-proxy",label,"_ep")}})}function assignRankMinMax(g){var maxRank=0;_.each(g.nodes(),function(v){var node=g.node(v);if(node.borderTop){node.minRank=g.node(node.borderTop).rank;node.maxRank=g.node(node.borderBottom).rank;maxRank=_.max(maxRank,node.maxRank)}});g.graph().maxRank=maxRank}function removeEdgeLabelProxies(g){_.each(g.nodes(),function(v){var node=g.node(v);if(node.dummy==="edge-proxy"){g.edge(node.e).labelRank=node.rank;g.removeNode(v)}})}function translateGraph(g){var minX=Number.POSITIVE_INFINITY,maxX=0,minY=Number.POSITIVE_INFINITY,maxY=0,graphLabel=g.graph(),marginX=graphLabel.marginx||0,marginY=graphLabel.marginy||0;function getExtremes(attrs){var x=attrs.x,y=attrs.y,w=attrs.width,h=attrs.height;minX=Math.min(minX,x-w/2);maxX=Math.max(maxX,x+w/2);minY=Math.min(minY,y-h/2);maxY=Math.max(maxY,y+h/2)}_.each(g.nodes(),function(v){getExtremes(g.node(v))});_.each(g.edges(),function(e){var edge=g.edge(e);if(_.has(edge,"x")){getExtremes(edge)}});minX-=marginX;minY-=marginY;_.each(g.nodes(),function(v){var node=g.node(v);node.x-=minX;node.y-=minY});_.each(g.edges(),function(e){var edge=g.edge(e);_.each(edge.points,function(p){p.x-=minX;p.y-=minY});if(_.has(edge,"x")){edge.x-=minX}if(_.has(edge,"y")){edge.y-=minY}});graphLabel.width=maxX-minX+marginX;graphLabel.height=maxY-minY+marginY}function assignNodeIntersects(g){_.each(g.edges(),function(e){var edge=g.edge(e),nodeV=g.node(e.v),nodeW=g.node(e.w),p1,p2;if(!edge.points){edge.points=[];p1=nodeW;p2=nodeV}else{p1=edge.points[0];p2=edge.points[edge.points.length-1]}edge.points.unshift(util.intersectRect(nodeV,p1));edge.points.push(util.intersectRect(nodeW,p2))})}function fixupEdgeLabelCoords(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(_.has(edge,"x")){if(edge.labelpos==="l"||edge.labelpos==="r"){edge.width-=edge.labeloffset}switch(edge.labelpos){case"l":edge.x-=edge.width/2+edge.labeloffset;break;case"r":edge.x+=edge.width/2+edge.labeloffset;break}}})}function reversePointsForReversedEdges(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.reversed){edge.points.reverse()}})}function removeBorderNodes(g){_.each(g.nodes(),function(v){if(g.children(v).length){var node=g.node(v),t=g.node(node.borderTop),b=g.node(node.borderBottom),l=g.node(_.last(node.borderLeft)),r=g.node(_.last(node.borderRight));node.width=Math.abs(r.x-l.x);node.height=Math.abs(b.y-t.y);node.x=l.x+node.width/2;node.y=t.y+node.height/2}});_.each(g.nodes(),function(v){if(g.node(v).dummy==="border"){g.removeNode(v)}})}function removeSelfEdges(g){_.each(g.edges(),function(e){if(e.v===e.w){var node=g.node(e.v);if(!node.selfEdges){node.selfEdges=[]}node.selfEdges.push({e:e,label:g.edge(e)});g.removeEdge(e)}})}function insertSelfEdges(g){var layers=util.buildLayerMatrix(g);_.each(layers,function(layer){var orderShift=0;_.each(layer,function(v,i){var node=g.node(v);node.order=i+orderShift;_.each(node.selfEdges,function(selfEdge){util.addDummyNode(g,"selfedge",{width:selfEdge.label.width,height:selfEdge.label.height,rank:node.rank,order:i+ ++orderShift,e:selfEdge.e,label:selfEdge.label},"_se")});delete node.selfEdges})})}function positionSelfEdges(g){_.each(g.nodes(),function(v){var node=g.node(v);if(node.dummy==="selfedge"){var selfNode=g.node(node.e.v),x=selfNode.x+selfNode.width/2,y=selfNode.y,dx=node.x-x,dy=selfNode.height/2;g.setEdge(node.e,node.label);g.removeNode(v);node.label.points=[{x:x+2*dx/3,y:y-dy},{x:x+5*dx/6,y:y-dy},{x:x+dx,y:y},{x:x+5*dx/6,y:y+dy},{x:x+2*dx/3,y:y+dy}];node.label.x=node.x;node.label.y=node.y}})}function selectNumberAttrs(obj,attrs){return _.mapValues(_.pick(obj,attrs),Number)}function canonicalize(attrs){var newAttrs={};_.each(attrs,function(v,k){newAttrs[k.toLowerCase()]=v});return newAttrs}},{"./acyclic":2,"./add-border-segments":3,"./coordinate-system":4,"./graphlib":7,"./lodash":10,"./nesting-graph":11,"./normalize":12,"./order":17,"./parent-dummy-chains":22,"./position":24,"./rank":26,"./util":29}],10:[function(require,module,exports){var lodash;if(typeof require==="function"){try{lodash=require("lodash")}catch(e){}}if(!lodash){lodash=window._}module.exports=lodash},{lodash:51}],11:[function(require,module,exports){var _=require("./lodash"),util=require("./util");module.exports={run:run,cleanup:cleanup};function run(g){var root=util.addDummyNode(g,"root",{},"_root"),depths=treeDepths(g),height=_.max(depths)-1,nodeSep=2*height+1;g.graph().nestingRoot=root;_.each(g.edges(),function(e){g.edge(e).minlen*=nodeSep});var weight=sumWeights(g)+1;_.each(g.children(),function(child){dfs(g,root,nodeSep,weight,height,depths,child)});g.graph().nodeRankFactor=nodeSep}function dfs(g,root,nodeSep,weight,height,depths,v){var children=g.children(v);if(!children.length){if(v!==root){g.setEdge(root,v,{weight:0,minlen:nodeSep})}return}var top=util.addBorderNode(g,"_bt"),bottom=util.addBorderNode(g,"_bb"),label=g.node(v);g.setParent(top,v);label.borderTop=top;g.setParent(bottom,v);label.borderBottom=bottom;_.each(children,function(child){dfs(g,root,nodeSep,weight,height,depths,child);var childNode=g.node(child),childTop=childNode.borderTop?childNode.borderTop:child,childBottom=childNode.borderBottom?childNode.borderBottom:child,thisWeight=childNode.borderTop?weight:2*weight,minlen=childTop!==childBottom?1:height-depths[v]+1;g.setEdge(top,childTop,{weight:thisWeight,minlen:minlen,nestingEdge:true});g.setEdge(childBottom,bottom,{weight:thisWeight,minlen:minlen,nestingEdge:true})});if(!g.parent(v)){g.setEdge(root,top,{weight:0,minlen:height+depths[v]})}}function treeDepths(g){var depths={};function dfs(v,depth){var children=g.children(v);if(children&&children.length){_.each(children,function(child){dfs(child,depth+1)})}depths[v]=depth}_.each(g.children(),function(v){dfs(v,1)});return depths}function sumWeights(g){return _.reduce(g.edges(),function(acc,e){return acc+g.edge(e).weight},0)}function cleanup(g){var graphLabel=g.graph();g.removeNode(graphLabel.nestingRoot);delete graphLabel.nestingRoot;_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.nestingEdge){g.removeEdge(e)}})}},{"./lodash":10,"./util":29}],12:[function(require,module,exports){"use strict";var _=require("./lodash"),util=require("./util");module.exports={run:run,undo:undo};function run(g){g.graph().dummyChains=[];_.each(g.edges(),function(edge){normalizeEdge(g,edge)})}function normalizeEdge(g,e){var v=e.v,vRank=g.node(v).rank,w=e.w,wRank=g.node(w).rank,name=e.name,edgeLabel=g.edge(e),labelRank=edgeLabel.labelRank;if(wRank===vRank+1)return;g.removeEdge(e);var dummy,attrs,i;for(i=0,++vRank;vRank0){if(index%2){weightSum+=tree[index+1]}index=index-1>>1;tree[index]+=entry.weight}cc+=entry.weight*weightSum}));return cc}},{"../lodash":10}],17:[function(require,module,exports){"use strict";var _=require("../lodash"),initOrder=require("./init-order"),crossCount=require("./cross-count"),sortSubgraph=require("./sort-subgraph"),buildLayerGraph=require("./build-layer-graph"),addSubgraphConstraints=require("./add-subgraph-constraints"),Graph=require("../graphlib").Graph,util=require("../util");module.exports=order;function order(g){var maxRank=util.maxRank(g),downLayerGraphs=buildLayerGraphs(g,_.range(1,maxRank+1),"inEdges"),upLayerGraphs=buildLayerGraphs(g,_.range(maxRank-1,-1,-1),"outEdges");var layering=initOrder(g);assignOrder(g,layering);var bestCC=Number.POSITIVE_INFINITY,best;for(var i=0,lastBest=0;lastBest<4;++i,++lastBest){sweepLayerGraphs(i%2?downLayerGraphs:upLayerGraphs,i%4>=2);layering=util.buildLayerMatrix(g);var cc=crossCount(g,layering);if(cc=vEntry.barycenter){mergeEntries(vEntry,uEntry)}}}function handleOut(vEntry){return function(wEntry){wEntry["in"].push(vEntry);if(--wEntry.indegree===0){sourceSet.push(wEntry)}}}while(sourceSet.length){var entry=sourceSet.pop();entries.push(entry);_.each(entry["in"].reverse(),handleIn(entry));_.each(entry.out,handleOut(entry))}return _.chain(entries).filter(function(entry){return!entry.merged}).map(function(entry){return _.pick(entry,["vs","i","barycenter","weight"])}).value()}function mergeEntries(target,source){var sum=0,weight=0;if(target.weight){sum+=target.barycenter*target.weight;weight+=target.weight}if(source.weight){sum+=source.barycenter*source.weight;weight+=source.weight}target.vs=source.vs.concat(target.vs);target.barycenter=sum/weight;target.weight=weight;target.i=Math.min(source.i,target.i);source.merged=true}},{"../lodash":10}],20:[function(require,module,exports){var _=require("../lodash"),barycenter=require("./barycenter"),resolveConflicts=require("./resolve-conflicts"),sort=require("./sort");module.exports=sortSubgraph;function sortSubgraph(g,v,cg,biasRight){var movable=g.children(v),node=g.node(v),bl=node?node.borderLeft:undefined,br=node?node.borderRight:undefined,subgraphs={};if(bl){movable=_.filter(movable,function(w){return w!==bl&&w!==br})}var barycenters=barycenter(g,movable);_.each(barycenters,function(entry){if(g.children(entry.v).length){var subgraphResult=sortSubgraph(g,entry.v,cg,biasRight);subgraphs[entry.v]=subgraphResult;if(_.has(subgraphResult,"barycenter")){mergeBarycenters(entry,subgraphResult)}}});var entries=resolveConflicts(barycenters,cg);expandSubgraphs(entries,subgraphs);var result=sort(entries,biasRight);if(bl){result.vs=_.flatten([bl,result.vs,br],true);if(g.predecessors(bl).length){var blPred=g.node(g.predecessors(bl)[0]),brPred=g.node(g.predecessors(br)[0]);if(!_.has(result,"barycenter")){result.barycenter=0;result.weight=0}result.barycenter=(result.barycenter*result.weight+blPred.order+brPred.order)/(result.weight+2);result.weight+=2}}return result}function expandSubgraphs(entries,subgraphs){_.each(entries,function(entry){entry.vs=_.flatten(entry.vs.map(function(v){if(subgraphs[v]){return subgraphs[v].vs}return v}),true)})}function mergeBarycenters(target,other){if(!_.isUndefined(target.barycenter)){target.barycenter=(target.barycenter*target.weight+other.barycenter*other.weight)/(target.weight+other.weight);target.weight+=other.weight}else{target.barycenter=other.barycenter;target.weight=other.weight}}},{"../lodash":10,"./barycenter":14,"./resolve-conflicts":19,"./sort":21}],21:[function(require,module,exports){var _=require("../lodash"),util=require("../util");module.exports=sort;function sort(entries,biasRight){var parts=util.partition(entries,function(entry){return _.has(entry,"barycenter")});var sortable=parts.lhs,unsortable=_.sortBy(parts.rhs,function(entry){return-entry.i}),vs=[],sum=0,weight=0,vsIndex=0;sortable.sort(compareWithBias(!!biasRight));vsIndex=consumeUnsortable(vs,unsortable,vsIndex);_.each(sortable,function(entry){vsIndex+=entry.vs.length;vs.push(entry.vs);sum+=entry.barycenter*entry.weight;weight+=entry.weight;vsIndex=consumeUnsortable(vs,unsortable,vsIndex)});var result={vs:_.flatten(vs,true)};if(weight){result.barycenter=sum/weight;result.weight=weight}return result}function consumeUnsortable(vs,unsortable,index){var last;while(unsortable.length&&(last=_.last(unsortable)).i<=index){unsortable.pop();vs.push(last.vs);index++}return index}function compareWithBias(bias){return function(entryV,entryW){if(entryV.barycenterentryW.barycenter){return 1}return!bias?entryV.i-entryW.i:entryW.i-entryV.i}}},{"../lodash":10,"../util":29}],22:[function(require,module,exports){var _=require("./lodash");module.exports=parentDummyChains;function parentDummyChains(g){var postorderNums=postorder(g);_.each(g.graph().dummyChains,function(v){var node=g.node(v),edgeObj=node.edgeObj,pathData=findPath(g,postorderNums,edgeObj.v,edgeObj.w),path=pathData.path,lca=pathData.lca,pathIdx=0,pathV=path[pathIdx],ascending=true;while(v!==edgeObj.w){node=g.node(v);if(ascending){while((pathV=path[pathIdx])!==lca&&g.node(pathV).maxRanklow||lim>postorderNums[parent].lim));lca=parent;parent=w;while((parent=g.parent(parent))!==lca){wPath.push(parent)}return{path:vPath.concat(wPath.reverse()),lca:lca}}function postorder(g){var result={},lim=0;function dfs(v){var low=lim;_.each(g.children(v),dfs);result[v]={low:low,lim:lim++}}_.each(g.children(),dfs);return result}},{"./lodash":10}],23:[function(require,module,exports){"use strict";var _=require("../lodash"),Graph=require("../graphlib").Graph,util=require("../util");module.exports={positionX:positionX,findType1Conflicts:findType1Conflicts,findType2Conflicts:findType2Conflicts,addConflict:addConflict,hasConflict:hasConflict,verticalAlignment:verticalAlignment,horizontalCompaction:horizontalCompaction,alignCoordinates:alignCoordinates,findSmallestWidthAlignment:findSmallestWidthAlignment,balance:balance};function findType1Conflicts(g,layering){var conflicts={};function visitLayer(prevLayer,layer){var k0=0,scanPos=0,prevLayerLength=prevLayer.length,lastNode=_.last(layer);_.each(layer,function(v,i){var w=findOtherInnerSegmentNode(g,v),k1=w?g.node(w).order:prevLayerLength;if(w||v===lastNode){_.each(layer.slice(scanPos,i+1),function(scanNode){_.each(g.predecessors(scanNode),function(u){var uLabel=g.node(u),uPos=uLabel.order; +if((uPosnextNorthBorder)){addConflict(conflicts,u,v)}})}})}function visitLayer(north,south){var prevNorthPos=-1,nextNorthPos,southPos=0;_.each(south,function(v,southLookahead){if(g.node(v).dummy==="border"){var predecessors=g.predecessors(v);if(predecessors.length){nextNorthPos=g.node(predecessors[0]).order;scan(south,southPos,southLookahead,prevNorthPos,nextNorthPos);southPos=southLookahead;prevNorthPos=nextNorthPos}}scan(south,southPos,south.length,nextNorthPos,north.length)});return south}_.reduce(layering,visitLayer);return conflicts}function findOtherInnerSegmentNode(g,v){if(g.node(v).dummy){return _.find(g.predecessors(v),function(u){return g.node(u).dummy})}}function addConflict(conflicts,v,w){if(v>w){var tmp=v;v=w;w=tmp}var conflictsV=conflicts[v];if(!conflictsV){conflicts[v]=conflictsV={}}conflictsV[w]=true}function hasConflict(conflicts,v,w){if(v>w){var tmp=v;v=w;w=tmp}return _.has(conflicts[v],w)}function verticalAlignment(g,layering,conflicts,neighborFn){var root={},align={},pos={};_.each(layering,function(layer){_.each(layer,function(v,order){root[v]=v;align[v]=v;pos[v]=order})});_.each(layering,function(layer){var prevIdx=-1;_.each(layer,function(v){var ws=neighborFn(v);if(ws.length){ws=_.sortBy(ws,function(w){return pos[w]});var mp=(ws.length-1)/2;for(var i=Math.floor(mp),il=Math.ceil(mp);i<=il;++i){var w=ws[i];if(align[v]===v&&prevIdxwLabel.lim){tailLabel=wLabel;flip=true}var candidates=_.filter(g.edges(),function(edge){return flip===isDescendant(t,t.node(edge.v),tailLabel)&&flip!==isDescendant(t,t.node(edge.w),tailLabel)});return _.min(candidates,function(edge){return slack(g,edge)})}function exchangeEdges(t,g,e,f){var v=e.v,w=e.w;t.removeEdge(v,w);t.setEdge(f.v,f.w,{});initLowLimValues(t);initCutValues(t,g);updateRanks(t,g)}function updateRanks(t,g){var root=_.find(t.nodes(),function(v){return!g.node(v).parent}),vs=preorder(t,root);vs=vs.slice(1);_.each(vs,function(v){var parent=t.node(v).parent,edge=g.edge(v,parent),flipped=false;if(!edge){edge=g.edge(parent,v);flipped=true}g.node(v).rank=g.node(parent).rank+(flipped?edge.minlen:-edge.minlen)})}function isTreeEdge(tree,u,v){return tree.hasEdge(u,v)}function isDescendant(tree,vLabel,rootLabel){return rootLabel.low<=vLabel.lim&&vLabel.lim<=rootLabel.lim}},{"../graphlib":7,"../lodash":10,"../util":29,"./feasible-tree":25,"./util":28}],28:[function(require,module,exports){"use strict";var _=require("../lodash");module.exports={longestPath:longestPath,slack:slack};function longestPath(g){var visited={};function dfs(v){var label=g.node(v);if(_.has(visited,v)){return label.rank}visited[v]=true;var rank=_.min(_.map(g.outEdges(v),function(e){return dfs(e.w)-g.edge(e).minlen}));if(rank===Number.POSITIVE_INFINITY){rank=0}return label.rank=rank}_.each(g.sources(),dfs)}function slack(g,e){return g.node(e.w).rank-g.node(e.v).rank-g.edge(e).minlen}},{"../lodash":10}],29:[function(require,module,exports){"use strict";var _=require("./lodash"),Graph=require("./graphlib").Graph;module.exports={addDummyNode:addDummyNode,simplify:simplify,asNonCompoundGraph:asNonCompoundGraph,successorWeights:successorWeights,predecessorWeights:predecessorWeights,intersectRect:intersectRect,buildLayerMatrix:buildLayerMatrix,normalizeRanks:normalizeRanks,removeEmptyRanks:removeEmptyRanks,addBorderNode:addBorderNode,maxRank:maxRank,partition:partition,time:time,notime:notime};function addDummyNode(g,type,attrs,name){var v;do{v=_.uniqueId(name)}while(g.hasNode(v));attrs.dummy=type;g.setNode(v,attrs);return v}function simplify(g){var simplified=(new Graph).setGraph(g.graph());_.each(g.nodes(),function(v){simplified.setNode(v,g.node(v))});_.each(g.edges(),function(e){var simpleLabel=simplified.edge(e.v,e.w)||{weight:0,minlen:1},label=g.edge(e);simplified.setEdge(e.v,e.w,{weight:simpleLabel.weight+label.weight,minlen:Math.max(simpleLabel.minlen,label.minlen)})});return simplified}function asNonCompoundGraph(g){var simplified=new Graph({multigraph:g.isMultigraph()}).setGraph(g.graph());_.each(g.nodes(),function(v){if(!g.children(v).length){simplified.setNode(v,g.node(v))}});_.each(g.edges(),function(e){simplified.setEdge(e,g.edge(e))});return simplified}function successorWeights(g){var weightMap=_.map(g.nodes(),function(v){var sucs={};_.each(g.outEdges(v),function(e){sucs[e.w]=(sucs[e.w]||0)+g.edge(e).weight});return sucs});return _.zipObject(g.nodes(),weightMap)}function predecessorWeights(g){var weightMap=_.map(g.nodes(),function(v){var preds={};_.each(g.inEdges(v),function(e){preds[e.v]=(preds[e.v]||0)+g.edge(e).weight});return preds});return _.zipObject(g.nodes(),weightMap)}function intersectRect(rect,point){var x=rect.x;var y=rect.y;var dx=point.x-x;var dy=point.y-y;var w=rect.width/2;var h=rect.height/2;if(!dx&&!dy){throw new Error("Not possible to find intersection inside of the rectangle")}var sx,sy;if(Math.abs(dy)*w>Math.abs(dx)*h){if(dy<0){h=-h}sx=h*dx/dy;sy=h}else{if(dx<0){w=-w}sx=w;sy=w*dy/dx}return{x:x+sx,y:y+sy}}function buildLayerMatrix(g){var layering=_.map(_.range(maxRank(g)+1),function(){return[]});_.each(g.nodes(),function(v){var node=g.node(v),rank=node.rank;if(!_.isUndefined(rank)){layering[rank][node.order]=v}});return layering}function normalizeRanks(g){var min=_.min(_.map(g.nodes(),function(v){return g.node(v).rank}));_.each(g.nodes(),function(v){var node=g.node(v);if(_.has(node,"rank")){node.rank-=min}})}function removeEmptyRanks(g){var offset=_.min(_.map(g.nodes(),function(v){return g.node(v).rank}));var layers=[];_.each(g.nodes(),function(v){var rank=g.node(v).rank-offset;if(!layers[rank]){layers[rank]=[]}layers[rank].push(v)});var delta=0,nodeRankFactor=g.graph().nodeRankFactor;_.each(layers,function(vs,i){if(_.isUndefined(vs)&&i%nodeRankFactor!==0){--delta}else if(delta){_.each(vs,function(v){g.node(v).rank+=delta})}})}function addBorderNode(g,prefix,rank,order){var node={width:0,height:0};if(arguments.length>=4){node.rank=rank;node.order=order}return addDummyNode(g,"border",node,prefix)}function maxRank(g){return _.max(_.map(g.nodes(),function(v){var rank=g.node(v).rank;if(!_.isUndefined(rank)){return rank}}))}function partition(collection,fn){var result={lhs:[],rhs:[]};_.each(collection,function(value){if(fn(value)){result.lhs.push(value)}else{result.rhs.push(value)}});return result}function time(name,fn){var start=_.now();try{return fn()}finally{console.log(name+" time: "+(_.now()-start)+"ms")}}function notime(name,fn){return fn()}},{"./graphlib":7,"./lodash":10}],30:[function(require,module,exports){module.exports="0.7.4"},{}],31:[function(require,module,exports){var lib=require("./lib");module.exports={Graph:lib.Graph,json:require("./lib/json"),alg:require("./lib/alg"),version:lib.version}},{"./lib":47,"./lib/alg":38,"./lib/json":48}],32:[function(require,module,exports){var _=require("../lodash");module.exports=components;function components(g){var visited={},cmpts=[],cmpt;function dfs(v){if(_.has(visited,v))return;visited[v]=true;cmpt.push(v);_.each(g.successors(v),dfs);_.each(g.predecessors(v),dfs)}_.each(g.nodes(),function(v){cmpt=[];dfs(v);if(cmpt.length){cmpts.push(cmpt)}});return cmpts}},{"../lodash":49}],33:[function(require,module,exports){var _=require("../lodash");module.exports=dfs;function dfs(g,vs,order){if(!_.isArray(vs)){vs=[vs]}var acc=[],visited={};_.each(vs,function(v){if(!g.hasNode(v)){throw new Error("Graph does not have node: "+v)}doDfs(g,v,order==="post",visited,acc)});return acc}function doDfs(g,v,postorder,visited,acc){if(!_.has(visited,v)){visited[v]=true;if(!postorder){acc.push(v)}_.each(g.neighbors(v),function(w){doDfs(g,w,postorder,visited,acc)});if(postorder){acc.push(v)}}}},{"../lodash":49}],34:[function(require,module,exports){var dijkstra=require("./dijkstra"),_=require("../lodash");module.exports=dijkstraAll;function dijkstraAll(g,weightFunc,edgeFunc){return _.transform(g.nodes(),function(acc,v){acc[v]=dijkstra(g,v,weightFunc,edgeFunc)},{})}},{"../lodash":49,"./dijkstra":35}],35:[function(require,module,exports){var _=require("../lodash"),PriorityQueue=require("../data/priority-queue");module.exports=dijkstra;var DEFAULT_WEIGHT_FUNC=_.constant(1);function dijkstra(g,source,weightFn,edgeFn){return runDijkstra(g,String(source),weightFn||DEFAULT_WEIGHT_FUNC,edgeFn||function(v){return g.outEdges(v)})}function runDijkstra(g,source,weightFn,edgeFn){var results={},pq=new PriorityQueue,v,vEntry;var updateNeighbors=function(edge){var w=edge.v!==v?edge.v:edge.w,wEntry=results[w],weight=weightFn(edge),distance=vEntry.distance+weight;if(weight<0){throw new Error("dijkstra does not allow negative edge weights. "+"Bad edge: "+edge+" Weight: "+weight)}if(distance0){v=pq.removeMin();vEntry=results[v];if(vEntry.distance===Number.POSITIVE_INFINITY){break}edgeFn(v).forEach(updateNeighbors)}return results}},{"../data/priority-queue":45,"../lodash":49}],36:[function(require,module,exports){var _=require("../lodash"),tarjan=require("./tarjan");module.exports=findCycles;function findCycles(g){return _.filter(tarjan(g),function(cmpt){return cmpt.length>1||cmpt.length===1&&g.hasEdge(cmpt[0],cmpt[0])})}},{"../lodash":49,"./tarjan":43}],37:[function(require,module,exports){var _=require("../lodash");module.exports=floydWarshall;var DEFAULT_WEIGHT_FUNC=_.constant(1);function floydWarshall(g,weightFn,edgeFn){return runFloydWarshall(g,weightFn||DEFAULT_WEIGHT_FUNC,edgeFn||function(v){return g.outEdges(v)})}function runFloydWarshall(g,weightFn,edgeFn){var results={},nodes=g.nodes();nodes.forEach(function(v){results[v]={};results[v][v]={distance:0};nodes.forEach(function(w){if(v!==w){results[v][w]={distance:Number.POSITIVE_INFINITY}}});edgeFn(v).forEach(function(edge){var w=edge.v===v?edge.w:edge.v,d=weightFn(edge);results[v][w]={distance:d,predecessor:v}})});nodes.forEach(function(k){var rowK=results[k];nodes.forEach(function(i){var rowI=results[i];nodes.forEach(function(j){var ik=rowI[k];var kj=rowK[j];var ij=rowI[j];var altDistance=ik.distance+kj.distance;if(altDistance0){v=pq.removeMin();if(_.has(parents,v)){result.setEdge(v,parents[v])}else if(init){throw new Error("Input graph is not connected: "+g)}else{init=true}g.nodeEdges(v).forEach(updateNeighbors)}return result}},{"../data/priority-queue":45,"../graph":46,"../lodash":49}],43:[function(require,module,exports){var _=require("../lodash");module.exports=tarjan;function tarjan(g){var index=0,stack=[],visited={},results=[];function dfs(v){var entry=visited[v]={onStack:true,lowlink:index,index:index++};stack.push(v);g.successors(v).forEach(function(w){if(!_.has(visited,w)){dfs(w);entry.lowlink=Math.min(entry.lowlink,visited[w].lowlink)}else if(visited[w].onStack){entry.lowlink=Math.min(entry.lowlink,visited[w].index)}});if(entry.lowlink===entry.index){var cmpt=[],w;do{w=stack.pop();visited[w].onStack=false;cmpt.push(w)}while(v!==w);results.push(cmpt)}}g.nodes().forEach(function(v){if(!_.has(visited,v)){dfs(v)}});return results}},{"../lodash":49}],44:[function(require,module,exports){var _=require("../lodash");module.exports=topsort;topsort.CycleException=CycleException;function topsort(g){var visited={},stack={},results=[];function visit(node){if(_.has(stack,node)){throw new CycleException}if(!_.has(visited,node)){stack[node]=true;visited[node]=true;_.each(g.predecessors(node),visit);delete stack[node];results.push(node)}}_.each(g.sinks(),visit);if(_.size(visited)!==g.nodeCount()){throw new CycleException}return results}function CycleException(){}},{"../lodash":49}],45:[function(require,module,exports){var _=require("../lodash");module.exports=PriorityQueue;function PriorityQueue(){this._arr=[];this._keyIndices={}}PriorityQueue.prototype.size=function(){return this._arr.length};PriorityQueue.prototype.keys=function(){return this._arr.map(function(x){return x.key})};PriorityQueue.prototype.has=function(key){return _.has(this._keyIndices,key)};PriorityQueue.prototype.priority=function(key){var index=this._keyIndices[key];if(index!==undefined){return this._arr[index].priority}};PriorityQueue.prototype.min=function(){if(this.size()===0){throw new Error("Queue underflow")}return this._arr[0].key};PriorityQueue.prototype.add=function(key,priority){var keyIndices=this._keyIndices;key=String(key);if(!_.has(keyIndices,key)){var arr=this._arr;var index=arr.length;keyIndices[key]=index;arr.push({key:key,priority:priority});this._decrease(index);return true}return false};PriorityQueue.prototype.removeMin=function(){this._swap(0,this._arr.length-1);var min=this._arr.pop();delete this._keyIndices[min.key];this._heapify(0);return min.key};PriorityQueue.prototype.decrease=function(key,priority){var index=this._keyIndices[key];if(priority>this._arr[index].priority){throw new Error("New priority is greater than current priority. "+"Key: "+key+" Old: "+this._arr[index].priority+" New: "+priority)}this._arr[index].priority=priority;this._decrease(index)};PriorityQueue.prototype._heapify=function(i){var arr=this._arr;var l=2*i,r=l+1,largest=i;if(l>1;if(arr[parent].priority1){this.setNode(v,value)}else{this.setNode(v)}},this);return this};Graph.prototype.setNode=function(v,value){if(_.has(this._nodes,v)){if(arguments.length>1){this._nodes[v]=value}return this}this._nodes[v]=arguments.length>1?value:this._defaultNodeLabelFn(v);if(this._isCompound){this._parent[v]=GRAPH_NODE;this._children[v]={};this._children[GRAPH_NODE][v]=true}this._in[v]={};this._preds[v]={};this._out[v]={};this._sucs[v]={};++this._nodeCount;return this};Graph.prototype.node=function(v){return this._nodes[v]};Graph.prototype.hasNode=function(v){return _.has(this._nodes,v)};Graph.prototype.removeNode=function(v){var self=this;if(_.has(this._nodes,v)){var removeEdge=function(e){self.removeEdge(self._edgeObjs[e])};delete this._nodes[v];if(this._isCompound){this._removeFromParentsChildList(v);delete this._parent[v];_.each(this.children(v),function(child){this.setParent(child)},this);delete this._children[v]}_.each(_.keys(this._in[v]),removeEdge);delete this._in[v];delete this._preds[v];_.each(_.keys(this._out[v]),removeEdge);delete this._out[v];delete this._sucs[v];--this._nodeCount}return this};Graph.prototype.setParent=function(v,parent){if(!this._isCompound){throw new Error("Cannot set parent in a non-compound graph")}if(_.isUndefined(parent)){parent=GRAPH_NODE}else{parent+="";for(var ancestor=parent;!_.isUndefined(ancestor);ancestor=this.parent(ancestor)){if(ancestor===v){throw new Error("Setting "+parent+" as parent of "+v+" would create create a cycle")}}this.setNode(parent)}this.setNode(v);this._removeFromParentsChildList(v);this._parent[v]=parent;this._children[parent][v]=true;return this};Graph.prototype._removeFromParentsChildList=function(v){delete this._children[this._parent[v]][v]};Graph.prototype.parent=function(v){if(this._isCompound){var parent=this._parent[v];if(parent!==GRAPH_NODE){return parent}}};Graph.prototype.children=function(v){if(_.isUndefined(v)){v=GRAPH_NODE}if(this._isCompound){var children=this._children[v];if(children){return _.keys(children)}}else if(v===GRAPH_NODE){return this.nodes()}else if(this.hasNode(v)){return[]}};Graph.prototype.predecessors=function(v){var predsV=this._preds[v];if(predsV){return _.keys(predsV)}};Graph.prototype.successors=function(v){var sucsV=this._sucs[v];if(sucsV){return _.keys(sucsV)}};Graph.prototype.neighbors=function(v){var preds=this.predecessors(v);if(preds){return _.union(preds,this.successors(v))}};Graph.prototype.setDefaultEdgeLabel=function(newDefault){if(!_.isFunction(newDefault)){newDefault=_.constant(newDefault)}this._defaultEdgeLabelFn=newDefault;return this};Graph.prototype.edgeCount=function(){return this._edgeCount};Graph.prototype.edges=function(){return _.values(this._edgeObjs)};Graph.prototype.setPath=function(vs,value){var self=this,args=arguments;_.reduce(vs,function(v,w){if(args.length>1){self.setEdge(v,w,value)}else{self.setEdge(v,w)}return w});return this};Graph.prototype.setEdge=function(){var v,w,name,value,valueSpecified=false;if(_.isPlainObject(arguments[0])){v=arguments[0].v;w=arguments[0].w;name=arguments[0].name;if(arguments.length===2){value=arguments[1];valueSpecified=true}}else{v=arguments[0];w=arguments[1];name=arguments[3];if(arguments.length>2){value=arguments[2];valueSpecified=true}}v=""+v;w=""+w;if(!_.isUndefined(name)){name=""+name}var e=edgeArgsToId(this._isDirected,v,w,name);if(_.has(this._edgeLabels,e)){if(valueSpecified){this._edgeLabels[e]=value}return this}if(!_.isUndefined(name)&&!this._isMultigraph){throw new Error("Cannot set a named edge when isMultigraph = false")}this.setNode(v);this.setNode(w);this._edgeLabels[e]=valueSpecified?value:this._defaultEdgeLabelFn(v,w,name);var edgeObj=edgeArgsToObj(this._isDirected,v,w,name);v=edgeObj.v;w=edgeObj.w;Object.freeze(edgeObj);this._edgeObjs[e]=edgeObj;incrementOrInitEntry(this._preds[w],v);incrementOrInitEntry(this._sucs[v],w);this._in[w][e]=edgeObj;this._out[v][e]=edgeObj;this._edgeCount++;return this};Graph.prototype.edge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name);return this._edgeLabels[e]};Graph.prototype.hasEdge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name);return _.has(this._edgeLabels,e)};Graph.prototype.removeEdge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name),edge=this._edgeObjs[e];if(edge){v=edge.v;w=edge.w;delete this._edgeLabels[e];delete this._edgeObjs[e];decrementOrRemoveEntry(this._preds[w],v);decrementOrRemoveEntry(this._sucs[v],w);delete this._in[w][e];delete this._out[v][e];this._edgeCount--}return this};Graph.prototype.inEdges=function(v,u){var inV=this._in[v];if(inV){var edges=_.values(inV);if(!u){return edges}return _.filter(edges,function(edge){return edge.v===u})}};Graph.prototype.outEdges=function(v,w){var outV=this._out[v];if(outV){var edges=_.values(outV);if(!w){return edges}return _.filter(edges,function(edge){return edge.w===w})}};Graph.prototype.nodeEdges=function(v,w){var inEdges=this.inEdges(v,w);if(inEdges){return inEdges.concat(this.outEdges(v,w))}};function incrementOrInitEntry(map,k){if(_.has(map,k)){map[k]++}else{map[k]=1}}function decrementOrRemoveEntry(map,k){if(!--map[k]){delete map[k]}}function edgeArgsToId(isDirected,v,w,name){if(!isDirected&&v>w){var tmp=v;v=w;w=tmp}return v+EDGE_KEY_DELIM+w+EDGE_KEY_DELIM+(_.isUndefined(name)?DEFAULT_EDGE_NAME:name)}function edgeArgsToObj(isDirected,v,w,name){if(!isDirected&&v>w){var tmp=v;v=w;w=tmp}var edgeObj={v:v,w:w};if(name){edgeObj.name=name}return edgeObj}function edgeObjToId(isDirected,edgeObj){return edgeArgsToId(isDirected,edgeObj.v,edgeObj.w,edgeObj.name)}},{"./lodash":49}],47:[function(require,module,exports){module.exports={Graph:require("./graph"),version:require("./version")}},{"./graph":46,"./version":50}],48:[function(require,module,exports){var _=require("./lodash"),Graph=require("./graph");module.exports={write:write,read:read};function write(g){var json={options:{directed:g.isDirected(),multigraph:g.isMultigraph(),compound:g.isCompound()},nodes:writeNodes(g),edges:writeEdges(g)}; +if(!_.isUndefined(g.graph())){json.value=_.clone(g.graph())}return json}function writeNodes(g){return _.map(g.nodes(),function(v){var nodeValue=g.node(v),parent=g.parent(v),node={v:v};if(!_.isUndefined(nodeValue)){node.value=nodeValue}if(!_.isUndefined(parent)){node.parent=parent}return node})}function writeEdges(g){return _.map(g.edges(),function(e){var edgeValue=g.edge(e),edge={v:e.v,w:e.w};if(!_.isUndefined(e.name)){edge.name=e.name}if(!_.isUndefined(edgeValue)){edge.value=edgeValue}return edge})}function read(json){var g=new Graph(json.options).setGraph(json.value);_.each(json.nodes,function(entry){g.setNode(entry.v,entry.value);if(entry.parent){g.setParent(entry.v,entry.parent)}});_.each(json.edges,function(entry){g.setEdge({v:entry.v,w:entry.w,name:entry.name},entry.value)});return g}},{"./graph":46,"./lodash":49}],49:[function(require,module,exports){module.exports=require(10)},{"/Users/cpettitt/projects/dagre/lib/lodash.js":10,lodash:51}],50:[function(require,module,exports){module.exports="1.0.5"},{}],51:[function(require,module,exports){(function(global){(function(){var undefined;var VERSION="3.10.0";var BIND_FLAG=1,BIND_KEY_FLAG=2,CURRY_BOUND_FLAG=4,CURRY_FLAG=8,CURRY_RIGHT_FLAG=16,PARTIAL_FLAG=32,PARTIAL_RIGHT_FLAG=64,ARY_FLAG=128,REARG_FLAG=256;var DEFAULT_TRUNC_LENGTH=30,DEFAULT_TRUNC_OMISSION="...";var HOT_COUNT=150,HOT_SPAN=16;var LARGE_ARRAY_SIZE=200;var LAZY_FILTER_FLAG=1,LAZY_MAP_FLAG=2;var FUNC_ERROR_TEXT="Expected a function";var PLACEHOLDER="__lodash_placeholder__";var argsTag="[object Arguments]",arrayTag="[object Array]",boolTag="[object Boolean]",dateTag="[object Date]",errorTag="[object Error]",funcTag="[object Function]",mapTag="[object Map]",numberTag="[object Number]",objectTag="[object Object]",regexpTag="[object RegExp]",setTag="[object Set]",stringTag="[object String]",weakMapTag="[object WeakMap]";var arrayBufferTag="[object ArrayBuffer]",float32Tag="[object Float32Array]",float64Tag="[object Float64Array]",int8Tag="[object Int8Array]",int16Tag="[object Int16Array]",int32Tag="[object Int32Array]",uint8Tag="[object Uint8Array]",uint8ClampedTag="[object Uint8ClampedArray]",uint16Tag="[object Uint16Array]",uint32Tag="[object Uint32Array]";var reEmptyStringLeading=/\b__p \+= '';/g,reEmptyStringMiddle=/\b(__p \+=) '' \+/g,reEmptyStringTrailing=/(__e\(.*?\)|\b__t\)) \+\n'';/g;var reEscapedHtml=/&(?:amp|lt|gt|quot|#39|#96);/g,reUnescapedHtml=/[&<>"'`]/g,reHasEscapedHtml=RegExp(reEscapedHtml.source),reHasUnescapedHtml=RegExp(reUnescapedHtml.source);var reEscape=/<%-([\s\S]+?)%>/g,reEvaluate=/<%([\s\S]+?)%>/g,reInterpolate=/<%=([\s\S]+?)%>/g;var reIsDeepProp=/\.|\[(?:[^[\]]*|(["'])(?:(?!\1)[^\n\\]|\\.)*?\1)\]/,reIsPlainProp=/^\w*$/,rePropName=/[^.[\]]+|\[(?:(-?\d+(?:\.\d+)?)|(["'])((?:(?!\2)[^\n\\]|\\.)*?)\2)\]/g;var reRegExpChars=/^[:!,]|[\\^$.*+?()[\]{}|\/]|(^[0-9a-fA-Fnrtuvx])|([\n\r\u2028\u2029])/g,reHasRegExpChars=RegExp(reRegExpChars.source);var reComboMark=/[\u0300-\u036f\ufe20-\ufe23]/g;var reEscapeChar=/\\(\\)?/g;var reEsTemplate=/\$\{([^\\}]*(?:\\.[^\\}]*)*)\}/g;var reFlags=/\w*$/;var reHasHexPrefix=/^0[xX]/;var reIsHostCtor=/^\[object .+?Constructor\]$/;var reIsUint=/^\d+$/;var reLatin1=/[\xc0-\xd6\xd8-\xde\xdf-\xf6\xf8-\xff]/g;var reNoMatch=/($^)/;var reUnescapedString=/['\n\r\u2028\u2029\\]/g;var reWords=function(){var upper="[A-Z\\xc0-\\xd6\\xd8-\\xde]",lower="[a-z\\xdf-\\xf6\\xf8-\\xff]+";return RegExp(upper+"+(?="+upper+lower+")|"+upper+"?"+lower+"|"+upper+"+|[0-9]+","g")}();var contextProps=["Array","ArrayBuffer","Date","Error","Float32Array","Float64Array","Function","Int8Array","Int16Array","Int32Array","Math","Number","Object","RegExp","Set","String","_","clearTimeout","isFinite","parseFloat","parseInt","setTimeout","TypeError","Uint8Array","Uint8ClampedArray","Uint16Array","Uint32Array","WeakMap"];var templateCounter=-1;var typedArrayTags={};typedArrayTags[float32Tag]=typedArrayTags[float64Tag]=typedArrayTags[int8Tag]=typedArrayTags[int16Tag]=typedArrayTags[int32Tag]=typedArrayTags[uint8Tag]=typedArrayTags[uint8ClampedTag]=typedArrayTags[uint16Tag]=typedArrayTags[uint32Tag]=true;typedArrayTags[argsTag]=typedArrayTags[arrayTag]=typedArrayTags[arrayBufferTag]=typedArrayTags[boolTag]=typedArrayTags[dateTag]=typedArrayTags[errorTag]=typedArrayTags[funcTag]=typedArrayTags[mapTag]=typedArrayTags[numberTag]=typedArrayTags[objectTag]=typedArrayTags[regexpTag]=typedArrayTags[setTag]=typedArrayTags[stringTag]=typedArrayTags[weakMapTag]=false;var cloneableTags={};cloneableTags[argsTag]=cloneableTags[arrayTag]=cloneableTags[arrayBufferTag]=cloneableTags[boolTag]=cloneableTags[dateTag]=cloneableTags[float32Tag]=cloneableTags[float64Tag]=cloneableTags[int8Tag]=cloneableTags[int16Tag]=cloneableTags[int32Tag]=cloneableTags[numberTag]=cloneableTags[objectTag]=cloneableTags[regexpTag]=cloneableTags[stringTag]=cloneableTags[uint8Tag]=cloneableTags[uint8ClampedTag]=cloneableTags[uint16Tag]=cloneableTags[uint32Tag]=true;cloneableTags[errorTag]=cloneableTags[funcTag]=cloneableTags[mapTag]=cloneableTags[setTag]=cloneableTags[weakMapTag]=false;var deburredLetters={"À":"A","Á":"A","Â":"A","Ã":"A","Ä":"A","Å":"A","à":"a","á":"a","â":"a","ã":"a","ä":"a","å":"a","Ç":"C","ç":"c","Ð":"D","ð":"d","È":"E","É":"E","Ê":"E","Ë":"E","è":"e","é":"e","ê":"e","ë":"e","Ì":"I","Í":"I","Î":"I","Ï":"I","ì":"i","í":"i","î":"i","ï":"i","Ñ":"N","ñ":"n","Ò":"O","Ó":"O","Ô":"O","Õ":"O","Ö":"O","Ø":"O","ò":"o","ó":"o","ô":"o","õ":"o","ö":"o","ø":"o","Ù":"U","Ú":"U","Û":"U","Ü":"U","ù":"u","ú":"u","û":"u","ü":"u","Ý":"Y","ý":"y","ÿ":"y","Æ":"Ae","æ":"ae","Þ":"Th","þ":"th","ß":"ss"};var htmlEscapes={"&":"&","<":"<",">":">",'"':""","'":"'","`":"`"};var htmlUnescapes={"&":"&","<":"<",">":">",""":'"',"'":"'","`":"`"};var objectTypes={"function":true,object:true};var regexpEscapes={0:"x30",1:"x31",2:"x32",3:"x33",4:"x34",5:"x35",6:"x36",7:"x37",8:"x38",9:"x39",A:"x41",B:"x42",C:"x43",D:"x44",E:"x45",F:"x46",a:"x61",b:"x62",c:"x63",d:"x64",e:"x65",f:"x66",n:"x6e",r:"x72",t:"x74",u:"x75",v:"x76",x:"x78"};var stringEscapes={"\\":"\\","'":"'","\n":"n","\r":"r","\u2028":"u2028","\u2029":"u2029"};var freeExports=objectTypes[typeof exports]&&exports&&!exports.nodeType&&exports;var freeModule=objectTypes[typeof module]&&module&&!module.nodeType&&module;var freeGlobal=freeExports&&freeModule&&typeof global=="object"&&global&&global.Object&&global;var freeSelf=objectTypes[typeof self]&&self&&self.Object&&self;var freeWindow=objectTypes[typeof window]&&window&&window.Object&&window;var moduleExports=freeModule&&freeModule.exports===freeExports&&freeExports;var root=freeGlobal||freeWindow!==(this&&this.window)&&freeWindow||freeSelf||this;function baseCompareAscending(value,other){if(value!==other){var valIsNull=value===null,valIsUndef=value===undefined,valIsReflexive=value===value;var othIsNull=other===null,othIsUndef=other===undefined,othIsReflexive=other===other;if(value>other&&!othIsNull||!valIsReflexive||valIsNull&&!othIsUndef&&othIsReflexive||valIsUndef&&othIsReflexive){return 1}if(value-1){}return index}function charsRightIndex(string,chars){var index=string.length;while(index--&&chars.indexOf(string.charAt(index))>-1){}return index}function compareAscending(object,other){return baseCompareAscending(object.criteria,other.criteria)||object.index-other.index}function compareMultiple(object,other,orders){var index=-1,objCriteria=object.criteria,othCriteria=other.criteria,length=objCriteria.length,ordersLength=orders.length;while(++index=ordersLength){return result}var order=orders[index];return result*(order==="asc"||order===true?1:-1)}}return object.index-other.index}function deburrLetter(letter){return deburredLetters[letter]}function escapeHtmlChar(chr){return htmlEscapes[chr]}function escapeRegExpChar(chr,leadingChar,whitespaceChar){if(leadingChar){chr=regexpEscapes[chr]}else if(whitespaceChar){chr=stringEscapes[chr]}return"\\"+chr}function escapeStringChar(chr){return"\\"+stringEscapes[chr]}function indexOfNaN(array,fromIndex,fromRight){var length=array.length,index=fromIndex+(fromRight?0:-1);while(fromRight?index--:++index=9&&charCode<=13)||charCode==32||charCode==160||charCode==5760||charCode==6158||charCode>=8192&&(charCode<=8202||charCode==8232||charCode==8233||charCode==8239||charCode==8287||charCode==12288||charCode==65279)}function replaceHolders(array,placeholder){var index=-1,length=array.length,resIndex=-1,result=[];while(++index>>1;var MAX_SAFE_INTEGER=9007199254740991;var metaMap=WeakMap&&new WeakMap;var realNames={};function lodash(value){if(isObjectLike(value)&&!isArray(value)&&!(value instanceof LazyWrapper)){if(value instanceof LodashWrapper){return value}if(hasOwnProperty.call(value,"__chain__")&&hasOwnProperty.call(value,"__wrapped__")){return wrapperClone(value)}}return new LodashWrapper(value)}function baseLodash(){}function LodashWrapper(value,chainAll,actions){this.__wrapped__=value;this.__actions__=actions||[];this.__chain__=!!chainAll}var support=lodash.support={};lodash.templateSettings={escape:reEscape,evaluate:reEvaluate,interpolate:reInterpolate,variable:"",imports:{_:lodash}};function LazyWrapper(value){this.__wrapped__=value;this.__actions__=[];this.__dir__=1;this.__filtered__=false;this.__iteratees__=[];this.__takeCount__=POSITIVE_INFINITY;this.__views__=[]}function lazyClone(){var result=new LazyWrapper(this.__wrapped__);result.__actions__=arrayCopy(this.__actions__);result.__dir__=this.__dir__;result.__filtered__=this.__filtered__;result.__iteratees__=arrayCopy(this.__iteratees__);result.__takeCount__=this.__takeCount__;result.__views__=arrayCopy(this.__views__);return result}function lazyReverse(){if(this.__filtered__){var result=new LazyWrapper(this);result.__dir__=-1;result.__filtered__=true}else{result=this.clone();result.__dir__*=-1}return result}function lazyValue(){var array=this.__wrapped__.value(),dir=this.__dir__,isArr=isArray(array),isRight=dir<0,arrLength=isArr?array.length:0,view=getView(0,arrLength,this.__views__),start=view.start,end=view.end,length=end-start,index=isRight?end:start-1,iteratees=this.__iteratees__,iterLength=iteratees.length,resIndex=0,takeCount=nativeMin(length,this.__takeCount__);if(!isArr||arrLength=LARGE_ARRAY_SIZE?createCache(values):null,valuesLength=values.length;if(cache){indexOf=cacheIndexOf;isCommon=false;values=cache}outer:while(++indexlength?0:length+start}end=end===undefined||end>length?length:+end||0;if(end<0){end+=length}length=start>end?0:end>>>0;start>>>=0;while(startlength?0:length+start}end=end===undefined||end>length?length:+end||0;if(end<0){end+=length}length=start>end?0:end-start>>>0;start>>>=0;var result=Array(length);while(++index=LARGE_ARRAY_SIZE,seen=isLarge?createCache():null,result=[];if(seen){indexOf=cacheIndexOf;isCommon=false}else{isLarge=false;seen=iteratee?[]:result}outer:while(++index>>1,computed=array[mid];if((retHighest?computed<=value:computed2?sources[length-2]:undefined,guard=length>2?sources[2]:undefined,thisArg=length>1?sources[length-1]:undefined;if(typeof customizer=="function"){customizer=bindCallback(customizer,thisArg,5);length-=2}else{customizer=typeof thisArg=="function"?thisArg:undefined;length-=customizer?1:0}if(guard&&isIterateeCall(sources[0],sources[1],guard)){customizer=length<3?undefined:customizer;length=1}while(++index-1?collection[index]:undefined}return baseFind(collection,predicate,eachFunc)}}function createFindIndex(fromRight){return function(array,predicate,thisArg){if(!(array&&array.length)){return-1}predicate=getCallback(predicate,thisArg,3);return baseFindIndex(array,predicate,fromRight)}}function createFindKey(objectFunc){return function(object,predicate,thisArg){predicate=getCallback(predicate,thisArg,3);return baseFind(object,predicate,objectFunc,true)}}function createFlow(fromRight){return function(){var wrapper,length=arguments.length,index=fromRight?length:-1,leftIndex=0,funcs=Array(length);while(fromRight?index--:++index=LARGE_ARRAY_SIZE){return wrapper.plant(value).value()}var index=0,result=length?funcs[index].apply(this,args):value;while(++index=length||!nativeIsFinite(length)){return""}var padLength=length-strLength;chars=chars==null?" ":chars+"";return repeat(chars,nativeCeil(padLength/chars.length)).slice(0,padLength)}function createPartialWrapper(func,bitmask,thisArg,partials){var isBind=bitmask&BIND_FLAG,Ctor=createCtorWrapper(func);function wrapper(){var argsIndex=-1,argsLength=arguments.length,leftIndex=-1,leftLength=partials.length,args=Array(leftLength+argsLength);while(++leftIndexarrLength)){return false}while(++index-1&&value%1==0&&value-1&&value%1==0&&value<=MAX_SAFE_INTEGER}function isStrictComparable(value){return value===value&&!isObject(value)}function mergeData(data,source){var bitmask=data[1],srcBitmask=source[1],newBitmask=bitmask|srcBitmask,isCommon=newBitmask0){if(++count>=HOT_COUNT){return key}}else{count=0}return baseSetData(key,value)}}();function shimKeys(object){var props=keysIn(object),propsLength=props.length,length=propsLength&&object.length;var allowIndexes=!!length&&isLength(length)&&(isArray(object)||isArguments(object));var index=-1,result=[];while(++index=120?createCache(othIndex&&value):null}var array=arrays[0],index=-1,length=array?array.length:0,seen=caches[0];outer:while(++index-1){splice.call(array,fromIndex,1)}}return array}var pullAt=restParam(function(array,indexes){indexes=baseFlatten(indexes);var result=baseAt(array,indexes);basePullAt(array,indexes.sort(baseCompareAscending));return result});function remove(array,predicate,thisArg){var result=[];if(!(array&&array.length)){return result}var index=-1,indexes=[],length=array.length;predicate=getCallback(predicate,thisArg,3);while(++index2?arrays[length-2]:undefined,thisArg=length>1?arrays[length-1]:undefined;if(length>2&&typeof iteratee=="function"){length-=2}else{iteratee=length>1&&typeof thisArg=="function"?(--length,thisArg):undefined;thisArg=undefined}arrays.length=length;return unzipWith(arrays,iteratee,thisArg)});function chain(value){var result=lodash(value);result.__chain__=true;return result}function tap(value,interceptor,thisArg){interceptor.call(thisArg,value);return value}function thru(value,interceptor,thisArg){return interceptor.call(thisArg,value)}function wrapperChain(){return chain(this)}function wrapperCommit(){return new LodashWrapper(this.value(),this.__chain__)}var wrapperConcat=restParam(function(values){values=baseFlatten(values);return this.thru(function(array){return arrayConcat(isArray(array)?array:[toObject(array)],values)})});function wrapperPlant(value){var result,parent=this;while(parent instanceof baseLodash){var clone=wrapperClone(parent);if(result){previous.__wrapped__=clone}else{result=clone}var previous=clone;parent=parent.__wrapped__}previous.__wrapped__=value;return result}function wrapperReverse(){var value=this.__wrapped__;var interceptor=function(value){return wrapped&&wrapped.__dir__<0?value:value.reverse()};if(value instanceof LazyWrapper){var wrapped=value;if(this.__actions__.length){wrapped=new LazyWrapper(this)}wrapped=wrapped.reverse();wrapped.__actions__.push({func:thru,args:[interceptor],thisArg:undefined});return new LodashWrapper(wrapped,this.__chain__)}return this.thru(interceptor)}function wrapperToString(){return this.value()+""}function wrapperValue(){return baseWrapperValue(this.__wrapped__,this.__actions__)}var at=restParam(function(collection,props){return baseAt(collection,baseFlatten(props))});var countBy=createAggregator(function(result,value,key){hasOwnProperty.call(result,key)?++result[key]:result[key]=1});function every(collection,predicate,thisArg){var func=isArray(collection)?arrayEvery:baseEvery;if(thisArg&&isIterateeCall(collection,predicate,thisArg)){predicate=undefined}if(typeof predicate!="function"||thisArg!==undefined){predicate=getCallback(predicate,thisArg,3) +}return func(collection,predicate)}function filter(collection,predicate,thisArg){var func=isArray(collection)?arrayFilter:baseFilter;predicate=getCallback(predicate,thisArg,3);return func(collection,predicate)}var find=createFind(baseEach);var findLast=createFind(baseEachRight,true);function findWhere(collection,source){return find(collection,baseMatches(source))}var forEach=createForEach(arrayEach,baseEach);var forEachRight=createForEach(arrayEachRight,baseEachRight);var groupBy=createAggregator(function(result,value,key){if(hasOwnProperty.call(result,key)){result[key].push(value)}else{result[key]=[value]}});function includes(collection,target,fromIndex,guard){var length=collection?getLength(collection):0;if(!isLength(length)){collection=values(collection);length=collection.length}if(typeof fromIndex!="number"||guard&&isIterateeCall(target,fromIndex,guard)){fromIndex=0}else{fromIndex=fromIndex<0?nativeMax(length+fromIndex,0):fromIndex||0}return typeof collection=="string"||!isArray(collection)&&isString(collection)?fromIndex<=length&&collection.indexOf(target,fromIndex)>-1:!!length&&getIndexOf(collection,target,fromIndex)>-1}var indexBy=createAggregator(function(result,value,key){result[key]=value});var invoke=restParam(function(collection,path,args){var index=-1,isFunc=typeof path=="function",isProp=isKey(path),result=isArrayLike(collection)?Array(collection.length):[];baseEach(collection,function(value){var func=isFunc?path:isProp&&value!=null?value[path]:undefined;result[++index]=func?func.apply(value,args):invokePath(value,path,args)});return result});function map(collection,iteratee,thisArg){var func=isArray(collection)?arrayMap:baseMap;iteratee=getCallback(iteratee,thisArg,3);return func(collection,iteratee)}var partition=createAggregator(function(result,value,key){result[key?0:1].push(value)},function(){return[[],[]]});function pluck(collection,path){return map(collection,property(path))}var reduce=createReduce(arrayReduce,baseEach);var reduceRight=createReduce(arrayReduceRight,baseEachRight);function reject(collection,predicate,thisArg){var func=isArray(collection)?arrayFilter:baseFilter;predicate=getCallback(predicate,thisArg,3);return func(collection,function(value,index,collection){return!predicate(value,index,collection)})}function sample(collection,n,guard){if(guard?isIterateeCall(collection,n,guard):n==null){collection=toIterable(collection);var length=collection.length;return length>0?collection[baseRandom(0,length-1)]:undefined}var index=-1,result=toArray(collection),length=result.length,lastIndex=length-1;n=nativeMin(n<0?0:+n||0,length);while(++index0){result=func.apply(this,arguments)}if(n<=1){func=undefined}return result}}var bind=restParam(function(func,thisArg,partials){var bitmask=BIND_FLAG;if(partials.length){var holders=replaceHolders(partials,bind.placeholder);bitmask|=PARTIAL_FLAG}return createWrapper(func,bitmask,thisArg,partials,holders)});var bindAll=restParam(function(object,methodNames){methodNames=methodNames.length?baseFlatten(methodNames):functions(object);var index=-1,length=methodNames.length;while(++indexwait){complete(trailingCall,maxTimeoutId)}else{timeoutId=setTimeout(delayed,remaining)}}function maxDelayed(){complete(trailing,timeoutId)}function debounced(){args=arguments;stamp=now();thisArg=this;trailingCall=trailing&&(timeoutId||!leading);if(maxWait===false){var leadingCall=leading&&!timeoutId}else{if(!maxTimeoutId&&!leading){lastCalled=stamp}var remaining=maxWait-(stamp-lastCalled),isCalled=remaining<=0||remaining>maxWait;if(isCalled){if(maxTimeoutId){maxTimeoutId=clearTimeout(maxTimeoutId)}lastCalled=stamp;result=func.apply(thisArg,args)}else if(!maxTimeoutId){maxTimeoutId=setTimeout(maxDelayed,remaining)}}if(isCalled&&timeoutId){timeoutId=clearTimeout(timeoutId)}else if(!timeoutId&&wait!==maxWait){timeoutId=setTimeout(delayed,wait)}if(leadingCall){isCalled=true;result=func.apply(thisArg,args)}if(isCalled&&!timeoutId&&!maxTimeoutId){args=thisArg=undefined}return result}debounced.cancel=cancel;return debounced}var defer=restParam(function(func,args){return baseDelay(func,1,args)});var delay=restParam(function(func,wait,args){return baseDelay(func,wait,args)});var flow=createFlow();var flowRight=createFlow(true);function memoize(func,resolver){if(typeof func!="function"||resolver&&typeof resolver!="function"){throw new TypeError(FUNC_ERROR_TEXT)}var memoized=function(){var args=arguments,key=resolver?resolver.apply(this,args):args[0],cache=memoized.cache;if(cache.has(key)){return cache.get(key)}var result=func.apply(this,args);memoized.cache=cache.set(key,result);return result};memoized.cache=new memoize.Cache;return memoized}var modArgs=restParam(function(func,transforms){transforms=baseFlatten(transforms);if(typeof func!="function"||!arrayEvery(transforms,baseIsFunction)){throw new TypeError(FUNC_ERROR_TEXT)}var length=transforms.length;return restParam(function(args){var index=nativeMin(args.length,length);while(index--){args[index]=transforms[index](args[index])}return func.apply(this,args)})});function negate(predicate){if(typeof predicate!="function"){throw new TypeError(FUNC_ERROR_TEXT)}return function(){return!predicate.apply(this,arguments)}}function once(func){return before(2,func)}var partial=createPartial(PARTIAL_FLAG);var partialRight=createPartial(PARTIAL_RIGHT_FLAG);var rearg=restParam(function(func,indexes){return createWrapper(func,REARG_FLAG,undefined,undefined,undefined,baseFlatten(indexes))});function restParam(func,start){if(typeof func!="function"){throw new TypeError(FUNC_ERROR_TEXT)}start=nativeMax(start===undefined?func.length-1:+start||0,0);return function(){var args=arguments,index=-1,length=nativeMax(args.length-start,0),rest=Array(length);while(++indexother}function gte(value,other){return value>=other}function isArguments(value){return isObjectLike(value)&&isArrayLike(value)&&hasOwnProperty.call(value,"callee")&&!propertyIsEnumerable.call(value,"callee")}var isArray=nativeIsArray||function(value){return isObjectLike(value)&&isLength(value.length)&&objToString.call(value)==arrayTag};function isBoolean(value){return value===true||value===false||isObjectLike(value)&&objToString.call(value)==boolTag}function isDate(value){return isObjectLike(value)&&objToString.call(value)==dateTag}function isElement(value){return!!value&&value.nodeType===1&&isObjectLike(value)&&!isPlainObject(value)}function isEmpty(value){if(value==null){return true}if(isArrayLike(value)&&(isArray(value)||isString(value)||isArguments(value)||isObjectLike(value)&&isFunction(value.splice))){return!value.length}return!keys(value).length}function isEqual(value,other,customizer,thisArg){customizer=typeof customizer=="function"?bindCallback(customizer,thisArg,3):undefined;var result=customizer?customizer(value,other):undefined;return result===undefined?baseIsEqual(value,other,customizer):!!result}function isError(value){return isObjectLike(value)&&typeof value.message=="string"&&objToString.call(value)==errorTag}function isFinite(value){return typeof value=="number"&&nativeIsFinite(value)}function isFunction(value){return isObject(value)&&objToString.call(value)==funcTag}function isObject(value){var type=typeof value;return!!value&&(type=="object"||type=="function")}function isMatch(object,source,customizer,thisArg){customizer=typeof customizer=="function"?bindCallback(customizer,thisArg,3):undefined;return baseIsMatch(object,getMatchData(source),customizer)}function isNaN(value){return isNumber(value)&&value!=+value}function isNative(value){if(value==null){return false}if(isFunction(value)){return reIsNative.test(fnToString.call(value))}return isObjectLike(value)&&reIsHostCtor.test(value)}function isNull(value){return value===null}function isNumber(value){return typeof value=="number"||isObjectLike(value)&&objToString.call(value)==numberTag}function isPlainObject(value){var Ctor;if(!(isObjectLike(value)&&objToString.call(value)==objectTag&&!isArguments(value))||!hasOwnProperty.call(value,"constructor")&&(Ctor=value.constructor,typeof Ctor=="function"&&!(Ctor instanceof Ctor))){return false}var result;baseForIn(value,function(subValue,key){result=key});return result===undefined||hasOwnProperty.call(value,result)}function isRegExp(value){return isObject(value)&&objToString.call(value)==regexpTag}function isString(value){return typeof value=="string"||isObjectLike(value)&&objToString.call(value)==stringTag}function isTypedArray(value){return isObjectLike(value)&&isLength(value.length)&&!!typedArrayTags[objToString.call(value)]}function isUndefined(value){return value===undefined}function lt(value,other){return value0;while(++index=nativeMin(start,end)&&value=0&&string.indexOf(target,position)==position}function escape(string){string=baseToString(string);return string&&reHasUnescapedHtml.test(string)?string.replace(reUnescapedHtml,escapeHtmlChar):string}function escapeRegExp(string){string=baseToString(string);return string&&reHasRegExpChars.test(string)?string.replace(reRegExpChars,escapeRegExpChar):string||"(?:)"}var kebabCase=createCompounder(function(result,word,index){return result+(index?"-":"")+word.toLowerCase()});function pad(string,length,chars){string=baseToString(string);length=+length;var strLength=string.length;if(strLength>=length||!nativeIsFinite(length)){return string}var mid=(length-strLength)/2,leftLength=nativeFloor(mid),rightLength=nativeCeil(mid);chars=createPadding("",rightLength,chars);return chars.slice(0,leftLength)+string+chars}var padLeft=createPadDir();var padRight=createPadDir(true);function parseInt(string,radix,guard){if(guard?isIterateeCall(string,radix,guard):radix==null){radix=0}else if(radix){radix=+radix}string=trim(string);return nativeParseInt(string,radix||(reHasHexPrefix.test(string)?16:10))}function repeat(string,n){var result="";string=baseToString(string);n=+n;if(n<1||!string||!nativeIsFinite(n)){return result}do{if(n%2){result+=string}n=nativeFloor(n/2);string+=string}while(n);return result}var snakeCase=createCompounder(function(result,word,index){return result+(index?"_":"")+word.toLowerCase()});var startCase=createCompounder(function(result,word,index){return result+(index?" ":"")+(word.charAt(0).toUpperCase()+word.slice(1))});function startsWith(string,target,position){string=baseToString(string);position=position==null?0:nativeMin(position<0?0:+position||0,string.length);return string.lastIndexOf(target,position)==position}function template(string,options,otherOptions){var settings=lodash.templateSettings;if(otherOptions&&isIterateeCall(string,options,otherOptions)){options=otherOptions=undefined}string=baseToString(string);options=assignWith(baseAssign({},otherOptions||options),settings,assignOwnDefaults);var imports=assignWith(baseAssign({},options.imports),settings.imports,assignOwnDefaults),importsKeys=keys(imports),importsValues=baseValues(imports,importsKeys);var isEscaping,isEvaluating,index=0,interpolate=options.interpolate||reNoMatch,source="__p += '";var reDelimiters=RegExp((options.escape||reNoMatch).source+"|"+interpolate.source+"|"+(interpolate===reInterpolate?reEsTemplate:reNoMatch).source+"|"+(options.evaluate||reNoMatch).source+"|$","g");var sourceURL="//# sourceURL="+("sourceURL"in options?options.sourceURL:"lodash.templateSources["+ ++templateCounter+"]")+"\n";string.replace(reDelimiters,function(match,escapeValue,interpolateValue,esTemplateValue,evaluateValue,offset){interpolateValue||(interpolateValue=esTemplateValue);source+=string.slice(index,offset).replace(reUnescapedString,escapeStringChar);if(escapeValue){isEscaping=true;source+="' +\n__e("+escapeValue+") +\n'"}if(evaluateValue){isEvaluating=true;source+="';\n"+evaluateValue+";\n__p += '"}if(interpolateValue){source+="' +\n((__t = ("+interpolateValue+")) == null ? '' : __t) +\n'"}index=offset+match.length;return match});source+="';\n";var variable=options.variable;if(!variable){source="with (obj) {\n"+source+"\n}\n"}source=(isEvaluating?source.replace(reEmptyStringLeading,""):source).replace(reEmptyStringMiddle,"$1").replace(reEmptyStringTrailing,"$1;");source="function("+(variable||"obj")+") {\n"+(variable?"":"obj || (obj = {});\n")+"var __t, __p = ''"+(isEscaping?", __e = _.escape":"")+(isEvaluating?", __j = Array.prototype.join;\n"+"function print() { __p += __j.call(arguments, '') }\n":";\n")+source+"return __p\n}";var result=attempt(function(){return Function(importsKeys,sourceURL+"return "+source).apply(undefined,importsValues)});result.source=source;if(isError(result)){throw result}return result}function trim(string,chars,guard){var value=string;string=baseToString(string);if(!string){return string}if(guard?isIterateeCall(value,chars,guard):chars==null){return string.slice(trimmedLeftIndex(string),trimmedRightIndex(string)+1)}chars=chars+"";return string.slice(charsLeftIndex(string,chars),charsRightIndex(string,chars)+1)}function trimLeft(string,chars,guard){var value=string;string=baseToString(string);if(!string){return string}if(guard?isIterateeCall(value,chars,guard):chars==null){return string.slice(trimmedLeftIndex(string))}return string.slice(charsLeftIndex(string,chars+""))}function trimRight(string,chars,guard){var value=string;string=baseToString(string);if(!string){return string}if(guard?isIterateeCall(value,chars,guard):chars==null){return string.slice(0,trimmedRightIndex(string)+1)}return string.slice(0,charsRightIndex(string,chars+"")+1)}function trunc(string,options,guard){if(guard&&isIterateeCall(string,options,guard)){options=undefined}var length=DEFAULT_TRUNC_LENGTH,omission=DEFAULT_TRUNC_OMISSION;if(options!=null){if(isObject(options)){var separator="separator"in options?options.separator:separator;length="length"in options?+options.length||0:length;omission="omission"in options?baseToString(options.omission):omission}else{length=+options||0}}string=baseToString(string);if(length>=string.length){return string}var end=length-omission.length;if(end<1){return omission}var result=string.slice(0,end);if(separator==null){return result+omission}if(isRegExp(separator)){if(string.slice(end).search(separator)){var match,newEnd,substring=string.slice(0,end);if(!separator.global){separator=RegExp(separator.source,(reFlags.exec(separator)||"")+"g")}separator.lastIndex=0;while(match=separator.exec(substring)){newEnd=match.index}result=result.slice(0,newEnd==null?end:newEnd)}}else if(string.indexOf(separator,end)!=end){var index=result.lastIndexOf(separator);if(index>-1){result=result.slice(0,index)}}return result+omission}function unescape(string){string=baseToString(string);return string&&reHasEscapedHtml.test(string)?string.replace(reEscapedHtml,unescapeHtmlChar):string}function words(string,pattern,guard){if(guard&&isIterateeCall(string,pattern,guard)){pattern=undefined}string=baseToString(string);return string.match(pattern||reWords)||[]}var attempt=restParam(function(func,args){try{return func.apply(undefined,args)}catch(e){return isError(e)?e:new Error(e)}});function callback(func,thisArg,guard){if(guard&&isIterateeCall(func,thisArg,guard)){thisArg=undefined}return isObjectLike(func)?matches(func):baseCallback(func,thisArg)}function constant(value){return function(){return value}}function identity(value){return value}function matches(source){return baseMatches(baseClone(source,true))}function matchesProperty(path,srcValue){return baseMatchesProperty(path,baseClone(srcValue,true))}var method=restParam(function(path,args){return function(object){return invokePath(object,path,args)}});var methodOf=restParam(function(object,args){return function(path){return invokePath(object,path,args)}});function mixin(object,source,options){if(options==null){var isObj=isObject(source),props=isObj?keys(source):undefined,methodNames=props&&props.length?baseFunctions(source,props):undefined;if(!(methodNames?methodNames.length:isObj)){methodNames=false;options=source;source=object;object=this}}if(!methodNames){methodNames=baseFunctions(source,keys(source))}var chain=true,index=-1,isFunc=isFunction(object),length=methodNames.length;if(options===false){chain=false}else if(isObject(options)&&"chain"in options){chain=options.chain}while(++index0||end<0)){return new LazyWrapper(result)}if(start<0){result=result.takeRight(-start)}else if(start){result=result.drop(start)}if(end!==undefined){end=+end||0;result=end<0?result.dropRight(-end):result.take(end-start)}return result};LazyWrapper.prototype.takeRightWhile=function(predicate,thisArg){return this.reverse().takeWhile(predicate,thisArg).reverse()};LazyWrapper.prototype.toArray=function(){return this.take(POSITIVE_INFINITY)};baseForOwn(LazyWrapper.prototype,function(func,methodName){var checkIteratee=/^(?:filter|map|reject)|While$/.test(methodName),retUnwrapped=/^(?:first|last)$/.test(methodName),lodashFunc=lodash[retUnwrapped?"take"+(methodName=="last"?"Right":""):methodName];if(!lodashFunc){return}lodash.prototype[methodName]=function(){var args=retUnwrapped?[1]:arguments,chainAll=this.__chain__,value=this.__wrapped__,isHybrid=!!this.__actions__.length,isLazy=value instanceof LazyWrapper,iteratee=args[0],useLazy=isLazy||isArray(value);if(useLazy&&checkIteratee&&typeof iteratee=="function"&&iteratee.length!=1){isLazy=useLazy=false}var interceptor=function(value){return retUnwrapped&&chainAll?lodashFunc(value,1)[0]:lodashFunc.apply(undefined,arrayPush([value],args))};var action={func:thru,args:[interceptor],thisArg:undefined},onlyLazy=isLazy&&!isHybrid;if(retUnwrapped&&!chainAll){if(onlyLazy){value=value.clone();value.__actions__.push(action);return func.call(value)}return lodashFunc.call(undefined,this.value())[0]}if(!retUnwrapped&&useLazy){value=onlyLazy?value:new LazyWrapper(this);var result=func.apply(value,args);result.__actions__.push(action);return new LodashWrapper(result,chainAll)}return this.thru(interceptor)}});arrayEach(["join","pop","push","replace","shift","sort","splice","split","unshift"],function(methodName){var func=(/^(?:replace|split)$/.test(methodName)?stringProto:arrayProto)[methodName],chainName=/^(?:push|sort|unshift)$/.test(methodName)?"tap":"thru",retUnwrapped=/^(?:join|pop|replace|shift)$/.test(methodName);lodash.prototype[methodName]=function(){var args=arguments;if(retUnwrapped&&!this.__chain__){return func.apply(this.value(),args)}return this[chainName](function(value){return func.apply(value,args)})}});baseForOwn(LazyWrapper.prototype,function(func,methodName){var lodashFunc=lodash[methodName];if(lodashFunc){var key=lodashFunc.name,names=realNames[key]||(realNames[key]=[]);names.push({name:methodName,func:lodashFunc})}});realNames[createHybridWrapper(undefined,BIND_KEY_FLAG).name]=[{name:"wrapper",func:undefined}];LazyWrapper.prototype.clone=lazyClone;LazyWrapper.prototype.reverse=lazyReverse;LazyWrapper.prototype.value=lazyValue;lodash.prototype.chain=wrapperChain;lodash.prototype.commit=wrapperCommit;lodash.prototype.concat=wrapperConcat;lodash.prototype.plant=wrapperPlant;lodash.prototype.reverse=wrapperReverse;lodash.prototype.toString=wrapperToString;lodash.prototype.run=lodash.prototype.toJSON=lodash.prototype.valueOf=lodash.prototype.value=wrapperValue;lodash.prototype.collect=lodash.prototype.map;lodash.prototype.head=lodash.prototype.first;lodash.prototype.select=lodash.prototype.filter;lodash.prototype.tail=lodash.prototype.rest;return lodash}var _=runInContext();if(typeof define=="function"&&typeof define.amd=="object"&&define.amd){root._=_;define(function(){return _})}else if(freeExports&&freeModule){if(moduleExports){(freeModule.exports=_)._=_}else{freeExports._=_}}else{root._=_}}).call(this)}).call(this,typeof global!=="undefined"?global:typeof self!=="undefined"?self:typeof window!=="undefined"?window:{})},{}]},{},[1])(1)}); \ No newline at end of file diff --git a/storm-core/src/ui/public/js/esprima.min.js b/storm-core/src/ui/public/js/esprima.min.js new file mode 100644 index 00000000000..d74b2081257 --- /dev/null +++ b/storm-core/src/ui/public/js/esprima.min.js @@ -0,0 +1,2 @@ +!function(e){"use strict";function t(e){return"0123456789".indexOf(e)>=0}function r(e){return"0123456789abcdefABCDEF".indexOf(e)>=0}function n(e){return e>="a"&&"z">=e||e>="A"&&"Z">=e}function i(e){return e>="0"&&"9">=e}function a(e){return" "===e||"\x0B"===e||"\f"===e||" "===e||" "===e||"\ufeff"===e}function o(e){return"\n"===e||"\r"===e||"\u2028"===e||"\u2029"===e}function u(e){return"$"===e||"_"===e||n(e)}function s(e){return u(e)||i(e)}function p(e){switch(e){case"break":case"case":case"catch":case"continue":case"debugger":case"default":case"delete":case"do":case"else":case"finally":case"for":case"function":case"if":case"in":case"instanceof":case"new":case"return":case"switch":case"this":case"throw":case"try":case"typeof":case"var":case"void":case"while":case"with":case"class":case"const":case"enum":case"export":case"extends":case"import":case"super":case"implements":case"interface":case"let":case"package":case"private":case"protected":case"public":case"static":case"yield":return!0}return!1}function f(){var e="\x00",t=be;return we>t&&(e=ge[t],be+=1),e}function c(){var e,t,r;for(t=!1,r=!1;we>be;)if(e=ge[be],r)f(),o(e)&&(r=!1);else if(t)f(),"*"===e&&(e=ge[be],"/"===e&&(f(),t=!1));else if("/"===e)if(e=ge[be+1],"/"===e)f(),f(),r=!0;else{if("*"!==e)break;f(),f(),t=!0}else if(a(e))f();else{if(!o(e))break;f()}}function l(){var e,t;if(e=ge[be],u(e)){for(t=f();we>be&&(e=ge[be],s(e));)t+=f();return 1===t.length?{type:Ee.Identifier,value:t}:p(t)?{type:Ee.Keyword,value:t}:"null"===t?{type:Ee.NullLiteral}:"true"===t||"false"===t?{type:Ee.BooleanLiteral,value:t}:{type:Ee.Identifier,value:t}}}function y(){var e,r,n,i=ge[be];return";"===i||"{"===i||"}"===i?(f(),{type:Ee.Punctuator,value:i}):","===i||"("===i||")"===i?(f(),{type:Ee.Punctuator,value:i}):(e=ge[be+1],"."!==i||t(e)?(r=ge[be+2],n=ge[be+3],">"===i&&">"===e&&">"===r&&"="===n?(f(),f(),f(),f(),{type:Ee.Punctuator,value:">>>="}):"="===i&&"="===e&&"="===r?(f(),f(),f(),{type:Ee.Punctuator,value:"==="}):"!"===i&&"="===e&&"="===r?(f(),f(),f(),{type:Ee.Punctuator,value:"!=="}):">"===i&&">"===e&&">"===r?(f(),f(),f(),{type:Ee.Punctuator,value:">>>"}):"<"===i&&"<"===e&&"="===r?(f(),f(),f(),{type:Ee.Punctuator,value:"<<="}):">"===i&&">"===e&&"="===r?(f(),f(),f(),{type:Ee.Punctuator,value:">>="}):"="===e&&"<>=!+-*%&|^/".indexOf(i)>=0?(f(),f(),{type:Ee.Punctuator,value:i+e}):i===e&&"+-<>&|".indexOf(i)>=0&&"+-<>&|".indexOf(e)>=0?(f(),f(),{type:Ee.Punctuator,value:i+e}):"[]<>+-*%&|^!~?:=/".indexOf(i)>=0?{type:Ee.Punctuator,value:f()}:void 0):{type:Ee.Punctuator,value:f()})}function d(){var e,n;if(n=ge[be],t(n)||"."===n){if(e="","."!==n){if(e=f(),n=ge[be],"x"===n||"X"===n){for(e+=f();we>be&&(n=ge[be],r(n));)e+=f();return{type:Ee.NumericLiteral,value:parseInt(e,16)}}for(;we>be&&(n=ge[be],t(n));)e+=f()}if("."===n)for(e+=f();we>be&&(n=ge[be],t(n));)e+=f();if("e"===n||"E"===n){if(e+=f(),n=ge[be],"+"!==n&&"-"!==n&&!t(n))throw n="character "+n,be>=we&&(n=""),new Error("Unexpected "+n+" after the exponent sign");for(e+=f();we>be&&(n=ge[be],t(n));)e+=f()}if("."===e)throw new Error("Expecting decimal digits after the dot sign");return{type:Ee.NumericLiteral,value:parseFloat(e)}}}function v(){var e,t,r="";if(e=ge[be],"'"===e||'"'===e){for(f();we>be;){if(t=f(),"undefined"==typeof t)throw new Error("Unterminated string");if(t===e)break;"\\"===t?(r+=t,r+=f()):r+=t}return{type:Ee.StringLiteral,value:r}}}function x(){var e,t="",r=!1;if(Ie=null,c(),e=ge[be],"/"===e){for(t=f();we>be;)if(e=f(),t+=e,r)"]"===e&&(r=!1);else{if("\\"===e&&(t+=f()),"/"===e)break;if("["===e&&(r=!0),o(e))throw new Error("Unexpected line terminator in a regular expression")}for(;we>be&&(e=ge[be],s(e));)t+=f();return t}}function m(){var e,r;if(Ie=null,c(),be>=we)return{type:Ee.EOF};if(r=y(),"undefined"!=typeof r)return r;if(e=ge[be],"'"===e||'"'===e)return v();if("."===e||t(e))return d();if(r=l(),"undefined"!=typeof r)return r;throw new Error("Unknown token from character "+f())}function h(){var e,t;return null!==Ie?Ie:(e=be,t=m(),be=e,Ie=t)}function E(e){var t;if(e.type===Ee.EOF)throw new Error("Unexpected ");throw t=e.value,t.length>10&&(t=t.substr(0,10)+"..."),new Error("Unexpected token "+t)}function S(e){var t=m();t.type===Ee.Punctuator&&t.value===e||E(t)}function g(e){var t=m();t.type===Ee.Keyword&&t.value===e||E(t)}function b(e){var t=h();return t.type===Ee.Punctuator&&t.value===e}function w(e){var t=h();return t.type===Ee.Keyword&&t.value===e}function I(){var e=h(),t=e.value;return e.type!==Ee.Punctuator?!1:"="===t||"*="===t||"/="===t||"%="===t||"+="===t||"-="===t||"<<="===t||">>="===t||">>>="===t||"&="===t||"^="===t||"|="===t}function L(){b(";")&&m()}function k(){var e,t=[];for(S("[");we>be;){if(b("]")){m();break}if(b(","))m(),t.push(e);else{if(t.push(X()),b("]")){m();break}S(",")}}return{type:Se.ArrayExpression,elements:t}}function F(){function e(e){return e===Ee.Identifier||e===Ee.StringLiteral||e===Ee.NumericLiteral}var t,r,n=[];for(S("{");we>be&&(t=m(),t.type!==Ee.Punctuator||"}"!==t.value);){if(e(t.type)?(r={},t.type===Ee.Identifier?r.key={type:Se.Identifier,name:t.value}:r.key={type:Se.Literal,value:t.value},S(":"),r.value=X(),n.push(r)):E(t),t=h(),t.type===Ee.Punctuator&&"}"===t.value){m();break}S(",")}return{type:Se.ObjectExpression,properties:n}}function B(){var e,t;return b("[")?k():b("{")?F():b("(")?(m(),t=Z(),S(")"),t.expression):w("function")?ve():w("this")?(m(),{type:Se.ThisExpression}):b("/")||b("/=")?{type:Se.Literal,value:x()}:(e=m(),e.type===Ee.Identifier?{type:Se.Identifier,name:e.value}:e.type===Ee.BooleanLiteral?{type:Se.Literal,value:"true"===e.value}:e.type===Ee.NullLiteral?{type:Se.Literal,value:null}:e.type===Ee.NumericLiteral?{type:Se.Literal,value:e.value}:e.type===Ee.StringLiteral?{type:Se.Literal,value:e.value}:void 0)}function P(){var e=[];if(S("("),!b(")"))for(;we>be&&(e.push(X()),!b(")"));)S(",");return S(")"),e}function C(){var e,t,r;for(e=B();we>be;)if(b(".")){if(m(),t=m(),t.type!==Ee.Identifier)throw new Error("Expecting an identifier after dot (.)");r={type:Se.Identifier,name:t.value},e={type:Se.MemberExpression,computed:!1,object:e,property:r}}else if(b("["))m(),r=Z(),r.type===Se.ExpressionStatement&&(r=r.expression),e={type:Se.MemberExpression,computed:!0,object:e,property:r},S("]");else{if(!b("("))break;e={type:Se.CallExpression,callee:e,arguments:P()}}return e}function O(){var e,t,r;return e=w("new"),e?(m(),t=O()):t=C(),b("(")&&(r=P()),e?("undefined"==typeof r&&(r=[]),t.type===Se.CallExpression&&(r=t.arguments,t=t.callee),{type:Se.NewExpression,callee:t,arguments:r}):"undefined"!=typeof r?{type:Se.CallExpression,callee:t,arguments:r}:t}function D(){var e=O();return(b("++")||b("--"))&&(e={type:Se.UpdateExpression,operator:m().value,argument:e,prefix:!1}),e}function U(){return b("++")||b("--")?{type:Se.UpdateExpression,operator:m().value,argument:U(),prefix:!0}:b("+")||b("-")||b("~")||b("!")?{type:Se.UnaryExpression,operator:m().value,argument:U()}:w("delete")||w("void")||w("typeof")?{type:Se.UnaryExpression,operator:m().value,argument:U()}:D()}function N(){for(var e=U();b("*")||b("/")||b("%");)e={type:Se.BinaryExpression,operator:m().value,left:e,right:U()};return e}function j(){for(var e=N();b("+")||b("-");)e={type:Se.BinaryExpression,operator:m().value,left:e,right:N()};return e}function T(){for(var e=j();b("<<")||b(">>")||b(">>>");)e={type:Se.BinaryExpression,operator:m().value,left:e,right:j()};return e}function W(){var e=T();return b("<")||b(">")||b("<=")||b(">=")?e={type:Se.BinaryExpression,operator:m().value,left:e,right:W()}:w("in")?(m(),e={type:Se.BinaryExpression,operator:"in",left:e,right:W()}):w("instanceof")&&(m(),e={type:Se.BinaryExpression,operator:"instanceof",left:e,right:W()}),e}function A(){for(var e=W();b("==")||b("!=")||b("===")||b("!==");)e={type:Se.BinaryExpression,operator:m().value,left:e,right:W()};return e}function q(){for(var e=A();b("&");)m(),e={type:Se.BinaryExpression,operator:"&",left:e,right:A()};return e}function z(){for(var e=q();b("|");)m(),e={type:Se.BinaryExpression,operator:"|",left:e,right:q()};return e}function K(){for(var e=z();b("^");)m(),e={type:Se.BinaryExpression,operator:"^",left:e,right:z()};return e}function V(){for(var e=K();b("&&");)m(),e={type:Se.LogicalExpression,operator:"&&",left:e,right:K()};return e}function M(){for(var e=V();b("||");)m(),e={type:Se.LogicalExpression,operator:"||",left:e,right:V()};return e}function R(){var e,t;return e=h(),t=M(),"undefined"==typeof t&&E(e),b("?")&&(m(),t={type:Se.ConditionalExpression,test:t},t.consequent=X(),S(":"),t.alternate=X()),t}function X(){var e=R();return I()&&(e={type:Se.AssignmentExpression,operator:m().value,left:e,right:X()}),e}function Z(){var e=X();if(b(","))for(e={type:Se.SequenceExpression,expressions:[e]};we>be&&b(",");)m(),e.expressions.push(X());return{type:Se.ExpressionStatement,expression:e}}function $(){for(var e,t=[];we>be&&!b("}")&&(e=ye(),"undefined"!=typeof e);)t.push(e);return t}function _(){var e;return S("{"),e=$(),S("}"),{type:Se.BlockStatement,body:e}}function G(){var e,t,r;if(e=m(),e.type!==Ee.Identifier)throw new Error("Expected an identifier");return t={type:Se.Identifier,name:e.value},r=null,b("=")&&(m(),r=X()),{id:t,init:r}}function H(){for(var e=[];we>be&&(e.push(G()),b(","));)m();return e}function J(){var e;return g("var"),e=H(),L(),{type:Se.VariableDeclaration,declarations:e,kind:"var"}}function Q(){var e;return g("let"),e=H(),L(),{type:Se.VariableDeclaration,declarations:e,kind:"let"}}function Y(){return S(";"),{type:Se.EmptyStatement}}function ee(){var e=Z();return L(),e}function te(){var e,t,r;return g("if"),S("("),e=Z().expression,S(")"),t=ye(),w("else")?(m(),r=ye()):r=null,{type:Se.IfStatement,test:e,consequent:t,alternate:r}}function re(){var e,t;return g("do"),e=ye(),g("while"),S("("),t=Z().expression,S(")"),L(),{type:Se.DoWhileStatement,body:e,test:t}}function ne(){var e,t;return g("while"),S("("),e=Z().expression,S(")"),t=ye(),{type:Se.WhileStatement,test:e,body:t}}function ie(){var e,t,r,n,i,a,o;return t=r=n=null,g("for"),S("("),b(";")?m():(w("var")||w("let")?(e=m().value,t={type:Se.VariableDeclaration,declarations:H(),kind:e},w("in")&&(m(),i=t,a=Z().expression,t=null)):t=Z().expression,"undefined"==typeof i&&(t.hasOwnProperty("operator")&&"in"===t.operator?(i=t.left,a=t.right,t=null):S(";"))),"undefined"==typeof i&&(b(";")||(r=Z().expression),S(";"),b(")")||(n=Z().expression)),S(")"),o=ye(),"undefined"==typeof i?{type:Se.ForStatement,init:t,test:r,update:n,body:o}:{type:Se.ForInStatement,left:i,right:a,body:o,each:!1}}function ae(){var e,t=null;return g("continue"),e=h(),e.type===Ee.Identifier&&(m(),t={type:Se.Identifier,name:e.value}),L(),{type:Se.ContinueStatement,label:t}}function oe(){var e,t=null;return g("break"),e=h(),e.type===Ee.Identifier&&(m(),t={type:Se.Identifier,name:e.value}),L(),{type:Se.BreakStatement,label:t}}function ue(){var e,t=null;return g("return"),b(";")||(e=h(),b("}")||e.type===Ee.EOF||(t=Z().expression)),L(),{type:Se.ReturnStatement,argument:t}}function se(){var e,t;return g("with"),S("("),e=Z().expression,S(")"),t=ye(),{type:Se.WithStatement,object:e,body:t}}function pe(){var e,t,r,n,i;if(g("switch"),S("("),e=Z().expression,S(")"),S("{"),b("}"))return m(),{type:Se.SwitchStatement,discriminant:e};for(t=[];we>be&&!b("}");){for(w("default")?(m(),r=null):(g("case"),r=Z().expression),S(":"),n=[];we>be&&!(b("}")||w("default")||w("case"))&&(i=ye(),"undefined"!=typeof i);)n.push(i);t.push({type:Se.SwitchCase,test:r,consequent:n})}return S("}"),{type:Se.SwitchStatement,discriminant:e,cases:t}}function fe(){var e,t=null;return g("throw"),b(";")||(e=h(),e.type!==Ee.EOF&&(t=Z().expression)),L(),{type:Se.ThrowStatement,argument:t}}function ce(){var e,t,r=[],n=null;return g("try"),e=_(),w("catch")&&(m(),S("("),b(")")||(t=Z().expression),S(")"),r.push({type:Se.CatchClause,param:t,guard:null,body:_()})),w("finally")&&(m(),n=_()),{type:Se.TryStatement,block:e,handlers:r,finalizer:n}}function le(){return g("debugger"),L(),{type:Se.DebuggerStatement}}function ye(){var e,t=h();if(t.type!==Ee.EOF){if(t.type===Ee.Punctuator)switch(t.value){case";":return Y();case"{":return _();case"(":return ee()}if(t.type===Ee.Keyword)switch(t.value){case"break":return oe();case"continue":return ae();case"debugger":return le();case"do":return re();case"for":return ie();case"if":return te();case"let":return Q();case"return":return ue();case"switch":return pe();case"throw":return fe();case"try":return ce();case"var":return J();case"while":return ne();case"with":return se()}return e=ee(),e.expression.type===Se.FunctionExpression&&null!==e.expression.id?{type:Se.FunctionDeclaration,id:e.expression.id,params:e.expression.params,body:e.expression.body}:e.expression.type===Se.Identifier&&b(":")?(m(),{type:Se.LabeledStatement,label:e.expression,body:ye()}):e}}function de(){var e,t,r=null,n=[];if(g("function"),e=m(),e.type!==Ee.Identifier&&E(e),r={type:Se.Identifier,name:e.value},S("("),!b(")"))for(;we>be&&(e=m(),e.type!==Ee.Identifier&&E(e),n.push({type:Se.Identifier,name:e.value}),!b(")"));)S(",");return S(")"),t=_(),{type:Se.FunctionDeclaration,id:r,params:n,body:t}}function ve(){var e,t,r=null,n=[];if(g("function"),b("(")||(e=m(),e.type!==Ee.Identifier&&E(e),r={type:Se.Identifier,name:e.value}),S("("),!b(")"))for(;we>be&&(e=m(),e.type!==Ee.Identifier&&E(e),n.push({type:Se.Identifier,name:e.value}),!b(")"));)S(",");return S(")"),t=_(),{type:Se.FunctionExpression,id:r,params:n,body:t}}function xe(){var e;return e=h(),e.type!==Ee.EOF?w("function")?de():ye():void 0}function me(){for(var e,t=[];we>be&&(e=xe(),"undefined"!=typeof e);)t.push(e);return t}function he(){return{type:Se.Program,body:me()}}var Ee,Se,ge,be,we,Ie;Ee={BooleanLiteral:1,EOF:2,Identifier:3,Keyword:4,NullLiteral:5,NumericLiteral:6,Punctuator:7,StringLiteral:8},Se={AssignmentExpression:"AssignmentExpression",ArrayExpression:"ArrayExpression",BlockStatement:"BlockStatement",BinaryExpression:"BinaryExpression",BreakStatement:"BreakStatement",CallExpression:"CallExpression",CatchClause:"CatchClause",ConditionalExpression:"ConditionalExpression",ContinueStatement:"ContinueStatement",DoWhileStatement:"DoWhileStatement",DebuggerStatement:"DebuggerStatement",EmptyStatement:"EmptyStatement",ExpressionStatement:"ExpressionStatement",ForStatement:"ForStatement",ForInStatement:"ForInStatement",FunctionDeclaration:"FunctionDeclaration",FunctionExpression:"FunctionExpression",Identifier:"Identifier",IfStatement:"IfStatement",Literal:"Literal",LabeledStatement:"LabeledStatement",LogicalExpression:"LogicalExpression",MemberExpression:"MemberExpression",NewExpression:"NewExpression",ObjectExpression:"ObjectExpression",Program:"Program",ReturnStatement:"ReturnStatement",SequenceExpression:"SequenceExpression",SwitchStatement:"SwitchStatement",SwitchCase:"SwitchCase",ThisExpression:"ThisExpression",ThrowStatement:"ThrowStatement",TryStatement:"TryStatement",UnaryExpression:"UnaryExpression",UpdateExpression:"UpdateExpression",VariableDeclaration:"VariableDeclaration",WhileStatement:"WhileStatement",WithStatement:"WithStatement"},"function"==typeof Object.freeze&&(Object.freeze(Ee),Object.freeze(Se)),e.parse=function(e){return ge=e,be=0,we=ge.length,Ie=null,he()},e.version="0.9.2"}("undefined"==typeof exports?esprima={}:exports); +//# sourceMappingURL=esprima.min.js.map \ No newline at end of file diff --git a/storm-core/src/ui/public/js/js-yaml.min.js b/storm-core/src/ui/public/js/js-yaml.min.js new file mode 100644 index 00000000000..211c64ce362 --- /dev/null +++ b/storm-core/src/ui/public/js/js-yaml.min.js @@ -0,0 +1,3 @@ +/* js-yaml 3.7.0 https://github.com/nodeca/js-yaml */ +!function(e){if("object"==typeof exports&&"undefined"!=typeof module)module.exports=e();else if("function"==typeof define&&define.amd)define([],e);else{var t;t="undefined"!=typeof window?window:"undefined"!=typeof global?global:"undefined"!=typeof self?self:this,t.jsyaml=e()}}(function(){return function e(t,n,i){function r(a,s){if(!n[a]){if(!t[a]){var c="function"==typeof require&&require;if(!s&&c)return c(a,!0);if(o)return o(a,!0);var u=new Error("Cannot find module '"+a+"'");throw u.code="MODULE_NOT_FOUND",u}var l=n[a]={exports:{}};t[a][0].call(l.exports,function(e){var n=t[a][1][e];return r(n?n:e)},l,l.exports,e,t,n,i)}return n[a].exports}for(var o="function"==typeof require&&require,a=0;ai&&" "!==e[h+1],h=o);else if(!l(a))return le;m=m&&p(a)}c=c||d&&o-h-1>i&&" "!==e[h+1]}return s||c?" "===e[0]&&n>9?le:c?ue:ce:m&&!r(e)?ae:se}function h(e,t,n,i){e.dump=function(){function r(t){return c(e,t)}if(0===t.length)return"''";if(!e.noCompatMode&&oe.indexOf(t)!==-1)return"'"+t+"'";var o=e.indent*Math.max(1,n),s=e.lineWidth===-1?-1:Math.max(Math.min(e.lineWidth,40),e.lineWidth-o),u=i||e.flowLevel>-1&&n>=e.flowLevel;switch(d(t,u,e.indent,s,r)){case ae:return t;case se:return"'"+t.replace(/'/g,"''")+"'";case ce:return"|"+m(t,e.indent)+g(a(t,o));case ue:return">"+m(t,e.indent)+g(a(y(t,s),o));case le:return'"'+v(t,s)+'"';default:throw new N("impossible error: invalid scalar style")}}()}function m(e,t){var n=" "===e[0]?String(t):"",i="\n"===e[e.length-1],r=i&&("\n"===e[e.length-2]||"\n"===e),o=r?"+":i?"":"-";return n+o+"\n"}function g(e){return"\n"===e[e.length-1]?e.slice(0,-1):e}function y(e,t){for(var n,i,r=/(\n+)([^\n]*)/g,o=function(){var n=e.indexOf("\n");return n=n!==-1?n:e.length,r.lastIndex=n,x(e.slice(0,n),t)}(),a="\n"===e[0]||" "===e[0];i=r.exec(e);){var s=i[1],c=i[2];n=" "===c[0],o+=s+(a||n||""===c?"":"\n")+x(c,t),a=n}return o}function x(e,t){if(""===e||" "===e[0])return e;for(var n,i,r=/ [^ ]/g,o=0,a=0,s=0,c="";n=r.exec(e);)s=n.index,s-o>t&&(i=a>o?a:s,c+="\n"+e.slice(o,i),o=i+1),a=s;return c+="\n",c+=e.length-o>t&&a>o?e.slice(o,a)+"\n"+e.slice(a+1):e.slice(o),c.slice(1)}function v(e){for(var t,n,i="",o=0;o1024&&(s+="? "),s+=e.dump+": ",j(e,t,a,!1,!1)&&(s+=e.dump,c+=s));e.tag=u,e.dump="{"+c+"}"}function C(e,t,n,i){var r,o,a,c,u,l,p="",f=e.tag,d=Object.keys(n);if(e.sortKeys===!0)d.sort();else if("function"==typeof e.sortKeys)d.sort(e.sortKeys);else if(e.sortKeys)throw new N("sortKeys must be a boolean or a function");for(r=0,o=d.length;r1024,u&&(l+=e.dump&&U===e.dump.charCodeAt(0)?"?":"? "),l+=e.dump,u&&(l+=s(e,t)),j(e,t+1,c,!0,u)&&(l+=e.dump&&U===e.dump.charCodeAt(0)?":":": ",l+=e.dump,p+=l));e.tag=f,e.dump=p||"{}"}function k(e,t,n){var i,r,o,a,s,c;for(r=n?e.explicitTypes:e.implicitTypes,o=0,a=r.length;o tag resolver accepts not "'+c+'" style');i=s.represent[c](t,c)}e.dump=i}return!0}return!1}function j(e,t,n,i,r,o){e.tag=null,e.dump=n,k(e,n,!1)||k(e,n,!0);var a=T.call(e.dump);i&&(i=e.flowLevel<0||e.flowLevel>t);var s,c,u="[object Object]"===a||"[object Array]"===a;if(u&&(s=e.duplicates.indexOf(n),c=s!==-1),(null!==e.tag&&"?"!==e.tag||c||2!==e.indent&&t>0)&&(r=!1),c&&e.usedDuplicates[s])e.dump="*ref_"+s;else{if(u&&c&&!e.usedDuplicates[s]&&(e.usedDuplicates[s]=!0),"[object Object]"===a)i&&0!==Object.keys(e.dump).length?(C(e,t,e.dump,r),c&&(e.dump="&ref_"+s+e.dump)):(w(e,t,e.dump),c&&(e.dump="&ref_"+s+" "+e.dump));else if("[object Array]"===a)i&&0!==e.dump.length?(b(e,t,e.dump,r),c&&(e.dump="&ref_"+s+e.dump)):(A(e,t,e.dump),c&&(e.dump="&ref_"+s+" "+e.dump));else{if("[object String]"!==a){if(e.skipInvalid)return!1;throw new N("unacceptable kind of an object to dump "+a)}"?"!==e.tag&&h(e,e.dump,t,o)}null!==e.tag&&"?"!==e.tag&&(e.dump="!<"+e.tag+"> "+e.dump)}return!0}function I(e,t){var n,i,r=[],o=[];for(S(e,r,o),n=0,i=o.length;n>10)+55296,(e-65536&1023)+56320)}function f(e,t){this.input=e,this.filename=t.filename||null,this.schema=t.schema||K,this.onWarning=t.onWarning||null,this.legacy=t.legacy||!1,this.json=t.json||!1,this.listener=t.listener||null,this.implicitTypes=this.schema.compiledImplicit,this.typeMap=this.schema.compiledTypeMap,this.length=e.length,this.position=0,this.line=0,this.lineStart=0,this.lineIndent=0,this.documents=[]}function d(e,t){return new P(t,new W(e.filename,e.input,e.position,e.line,e.position-e.lineStart))}function h(e,t){throw d(e,t)}function m(e,t){e.onWarning&&e.onWarning.call(null,d(e,t))}function g(e,t,n,i){var r,o,a,s;if(t1&&(e.result+=R.repeat("\n",t-1))}function C(e,t,n){var s,c,u,l,p,f,d,h,m,y=e.kind,x=e.result;if(m=e.input.charCodeAt(e.position),o(m)||a(m)||35===m||38===m||42===m||33===m||124===m||62===m||39===m||34===m||37===m||64===m||96===m)return!1;if((63===m||45===m)&&(c=e.input.charCodeAt(e.position+1),o(c)||n&&a(c)))return!1;for(e.kind="scalar",e.result="",u=l=e.position,p=!1;0!==m;){if(58===m){if(c=e.input.charCodeAt(e.position+1),o(c)||n&&a(c))break}else if(35===m){if(s=e.input.charCodeAt(e.position-1),o(s))break}else{if(e.position===e.lineStart&&b(e)||n&&a(m))break;if(i(m)){if(f=e.line,d=e.lineStart,h=e.lineIndent,A(e,!1,-1),e.lineIndent>=t){p=!0,m=e.input.charCodeAt(e.position);continue}e.position=l,e.line=f,e.lineStart=d,e.lineIndent=h;break}}p&&(g(e,u,l,!1),w(e,e.line-f),u=l=e.position,p=!1),r(m)||(l=e.position+1),m=e.input.charCodeAt(++e.position)}return g(e,u,l,!1),!!e.result||(e.kind=y,e.result=x,!1)}function k(e,t){var n,r,o;if(n=e.input.charCodeAt(e.position),39!==n)return!1;for(e.kind="scalar",e.result="",e.position++,r=o=e.position;0!==(n=e.input.charCodeAt(e.position));)if(39===n){if(g(e,r,e.position,!0),n=e.input.charCodeAt(++e.position),39!==n)return!0;r=e.position,e.position++,o=e.position}else i(n)?(g(e,r,o,!0),w(e,A(e,!1,t)),r=o=e.position):e.position===e.lineStart&&b(e)?h(e,"unexpected end of the document within a single quoted scalar"):(e.position++,o=e.position);h(e,"unexpected end of the stream within a single quoted scalar")}function j(e,t){var n,r,o,a,u,l;if(l=e.input.charCodeAt(e.position),34!==l)return!1;for(e.kind="scalar",e.result="",e.position++,n=r=e.position;0!==(l=e.input.charCodeAt(e.position));){if(34===l)return g(e,n,e.position,!0),e.position++,!0;if(92===l){if(g(e,n,e.position,!0),l=e.input.charCodeAt(++e.position),i(l))A(e,!1,t);else if(l<256&&re[l])e.result+=oe[l],e.position++;else if((u=c(l))>0){for(o=u,a=0;o>0;o--)l=e.input.charCodeAt(++e.position),(u=s(l))>=0?a=(a<<4)+u:h(e,"expected hexadecimal character");e.result+=p(a),e.position++}else h(e,"unknown escape sequence");n=r=e.position}else i(l)?(g(e,n,r,!0),w(e,A(e,!1,t)),n=r=e.position):e.position===e.lineStart&&b(e)?h(e,"unexpected end of the document within a double quoted scalar"):(e.position++,r=e.position)}h(e,"unexpected end of the stream within a double quoted scalar")}function I(e,t){var n,i,r,a,s,c,u,l,p,f,d,m=!0,g=e.tag,y=e.anchor,v={};if(d=e.input.charCodeAt(e.position),91===d)a=93,u=!1,i=[];else{if(123!==d)return!1;a=125,u=!0,i={}}for(null!==e.anchor&&(e.anchorMap[e.anchor]=i),d=e.input.charCodeAt(++e.position);0!==d;){if(A(e,!0,t),d=e.input.charCodeAt(e.position),d===a)return e.position++,e.tag=g,e.anchor=y,e.kind=u?"mapping":"sequence",e.result=i,!0;m||h(e,"missed comma between flow collection entries"),p=l=f=null,s=c=!1,63===d&&(r=e.input.charCodeAt(e.position+1),o(r)&&(s=c=!0,e.position++,A(e,!0,t))),n=e.line,_(e,t,H,!1,!0),p=e.tag,l=e.result,A(e,!0,t),d=e.input.charCodeAt(e.position),!c&&e.line!==n||58!==d||(s=!0,d=e.input.charCodeAt(++e.position),A(e,!0,t),_(e,t,H,!1,!0),f=e.result),u?x(e,i,v,p,l,f):s?i.push(x(e,null,v,p,l,f)):i.push(l),A(e,!0,t),d=e.input.charCodeAt(e.position),44===d?(m=!0,d=e.input.charCodeAt(++e.position)):m=!1}h(e,"unexpected end of the stream within a flow collection")}function S(e,t){var n,o,a,s,c=z,l=!1,p=!1,f=t,d=0,m=!1;if(s=e.input.charCodeAt(e.position),124===s)o=!1;else{if(62!==s)return!1;o=!0}for(e.kind="scalar",e.result="";0!==s;)if(s=e.input.charCodeAt(++e.position),43===s||45===s)z===c?c=43===s?Q:J:h(e,"repeat of a chomping mode identifier");else{if(!((a=u(s))>=0))break;0===a?h(e,"bad explicit indentation width of a block scalar; it cannot be less than one"):p?h(e,"repeat of an indentation width identifier"):(f=t+a-1,p=!0)}if(r(s)){do s=e.input.charCodeAt(++e.position);while(r(s));if(35===s)do s=e.input.charCodeAt(++e.position);while(!i(s)&&0!==s)}for(;0!==s;){for(v(e),e.lineIndent=0,s=e.input.charCodeAt(e.position);(!p||e.lineIndentf&&(f=e.lineIndent),i(s))d++;else{if(e.lineIndentt)&&0!==r)h(e,"bad indentation of a sequence entry");else if(e.lineIndentt)&&(_(e,t,Z,!0,a)&&(y?m=e.result:g=e.result),y||(x(e,p,f,d,m,g),d=m=g=null),A(e,!0,-1),c=e.input.charCodeAt(e.position)),e.lineIndent>t&&0!==c)h(e,"bad indentation of a mapping entry");else if(e.lineIndentt?d=1:e.lineIndent===t?d=0:e.lineIndentt?d=1:e.lineIndent===t?d=0:e.lineIndent tag; it should be "'+l.kind+'", not "'+e.kind+'"'),l.resolve(e.result)?(e.result=l.construct(e.result),null!==e.anchor&&(e.anchorMap[e.anchor]=e.result)):h(e,"cannot resolve a node with !<"+e.tag+"> explicit tag")):h(e,"unknown tag !<"+e.tag+">");return null!==e.listener&&e.listener("close",e),null!==e.tag||null!==e.anchor||g}function T(e){var t,n,a,s,c=e.position,u=!1;for(e.version=null,e.checkLineBreaks=e.legacy,e.tagMap={},e.anchorMap={};0!==(s=e.input.charCodeAt(e.position))&&(A(e,!0,-1),s=e.input.charCodeAt(e.position),!(e.lineIndent>0||37!==s));){for(u=!0,s=e.input.charCodeAt(++e.position),t=e.position;0!==s&&!o(s);)s=e.input.charCodeAt(++e.position);for(n=e.input.slice(t,e.position),a=[],n.length<1&&h(e,"directive name must not be less than one character in length");0!==s;){for(;r(s);)s=e.input.charCodeAt(++e.position);if(35===s){do s=e.input.charCodeAt(++e.position);while(0!==s&&!i(s));break}if(i(s))break;for(t=e.position;0!==s&&!o(s);)s=e.input.charCodeAt(++e.position);a.push(e.input.slice(t,e.position))}0!==s&&v(e),$.call(se,n)?se[n](e,n,a):m(e,'unknown document directive "'+n+'"')}return A(e,!0,-1),0===e.lineIndent&&45===e.input.charCodeAt(e.position)&&45===e.input.charCodeAt(e.position+1)&&45===e.input.charCodeAt(e.position+2)?(e.position+=3,A(e,!0,-1)):u&&h(e,"directives end mark is expected"),_(e,e.lineIndent-1,Z,!1,!0),A(e,!0,-1),e.checkLineBreaks&&ee.test(e.input.slice(c,e.position))&&m(e,"non-ASCII line breaks are interpreted as content"),e.documents.push(e.result),e.position===e.lineStart&&b(e)?void(46===e.input.charCodeAt(e.position)&&(e.position+=3,A(e,!0,-1))):void(e.position0&&"\0\r\n…\u2028\u2029".indexOf(this.buffer.charAt(i-1))===-1;)if(i-=1,this.position-i>t/2-1){n=" ... ",i+=5;break}for(o="",a=this.position;at/2-1){o=" ... ",a-=5;break}return s=this.buffer.slice(i,a),r.repeat(" ",e)+n+s+o+"\n"+r.repeat(" ",e+this.position-i+n.length)+"^"},i.prototype.toString=function(e){var t,n="";return this.name&&(n+='in "'+this.name+'" '),n+="at line "+(this.line+1)+", column "+(this.column+1),e||(t=this.getSnippet(),t&&(n+=":\n"+t)),n},t.exports=i},{"./common":2}],7:[function(e,t,n){"use strict";function i(e,t,n){var r=[];return e.include.forEach(function(e){n=i(e,t,n)}),e[t].forEach(function(e){n.forEach(function(t,n){t.tag===e.tag&&t.kind===e.kind&&r.push(n)}),n.push(e)}),n.filter(function(e,t){return r.indexOf(t)===-1})}function r(){function e(e){i[e.kind][e.tag]=i.fallback[e.tag]=e}var t,n,i={scalar:{},sequence:{},mapping:{},fallback:{}};for(t=0,n=arguments.length;t64)){if(t<0)return!1;i+=6}return i%8===0}function r(e){var t,n,i=e.replace(/[\r\n=]/g,""),r=i.length,o=l,a=0,c=[];for(t=0;t>16&255),c.push(a>>8&255),c.push(255&a)),a=a<<6|o.indexOf(i.charAt(t));return n=r%4*6,0===n?(c.push(a>>16&255),c.push(a>>8&255),c.push(255&a)):18===n?(c.push(a>>10&255),c.push(a>>2&255)):12===n&&c.push(a>>4&255),s?new s(c):c}function o(e){var t,n,i="",r=0,o=e.length,a=l;for(t=0;t>18&63],i+=a[r>>12&63],i+=a[r>>6&63],i+=a[63&r]),r=(r<<8)+e[t];return n=o%3,0===n?(i+=a[r>>18&63],i+=a[r>>12&63],i+=a[r>>6&63],i+=a[63&r]):2===n?(i+=a[r>>10&63],i+=a[r>>4&63], +i+=a[r<<2&63],i+=a[64]):1===n&&(i+=a[r>>2&63],i+=a[r<<4&63],i+=a[64],i+=a[64]),i}function a(e){return s&&s.isBuffer(e)}var s;try{var c=e;s=c("buffer").Buffer}catch(e){}var u=e("../type"),l="ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789+/=\n\r";t.exports=new u("tag:yaml.org,2002:binary",{kind:"scalar",resolve:i,construct:r,predicate:a,represent:o})},{"../type":13}],15:[function(e,t,n){"use strict";function i(e){if(null===e)return!1;var t=e.length;return 4===t&&("true"===e||"True"===e||"TRUE"===e)||5===t&&("false"===e||"False"===e||"FALSE"===e)}function r(e){return"true"===e||"True"===e||"TRUE"===e}function o(e){return"[object Boolean]"===Object.prototype.toString.call(e)}var a=e("../type");t.exports=new a("tag:yaml.org,2002:bool",{kind:"scalar",resolve:i,construct:r,predicate:o,represent:{lowercase:function(e){return e?"true":"false"},uppercase:function(e){return e?"TRUE":"FALSE"},camelcase:function(e){return e?"True":"False"}},defaultStyle:"lowercase"})},{"../type":13}],16:[function(e,t,n){"use strict";function i(e){return null!==e&&!!u.test(e)}function r(e){var t,n,i,r;return t=e.replace(/_/g,"").toLowerCase(),n="-"===t[0]?-1:1,r=[],"+-".indexOf(t[0])>=0&&(t=t.slice(1)),".inf"===t?1===n?Number.POSITIVE_INFINITY:Number.NEGATIVE_INFINITY:".nan"===t?NaN:t.indexOf(":")>=0?(t.split(":").forEach(function(e){r.unshift(parseFloat(e,10))}),t=0,i=1,r.forEach(function(e){t+=e*i,i*=60}),n*t):n*parseFloat(t,10)}function o(e,t){var n;if(isNaN(e))switch(t){case"lowercase":return".nan";case"uppercase":return".NAN";case"camelcase":return".NaN"}else if(Number.POSITIVE_INFINITY===e)switch(t){case"lowercase":return".inf";case"uppercase":return".INF";case"camelcase":return".Inf"}else if(Number.NEGATIVE_INFINITY===e)switch(t){case"lowercase":return"-.inf";case"uppercase":return"-.INF";case"camelcase":return"-.Inf"}else if(s.isNegativeZero(e))return"-0.0";return n=e.toString(10),l.test(n)?n.replace("e",".e"):n}function a(e){return"[object Number]"===Object.prototype.toString.call(e)&&(e%1!==0||s.isNegativeZero(e))}var s=e("../common"),c=e("../type"),u=new RegExp("^(?:[-+]?(?:[0-9][0-9_]*)\\.[0-9_]*(?:[eE][-+][0-9]+)?|\\.[0-9_]+(?:[eE][-+][0-9]+)?|[-+]?[0-9][0-9_]*(?::[0-5]?[0-9])+\\.[0-9_]*|[-+]?\\.(?:inf|Inf|INF)|\\.(?:nan|NaN|NAN))$"),l=/^[-+]?[0-9]+e/;t.exports=new c("tag:yaml.org,2002:float",{kind:"scalar",resolve:i,construct:r,predicate:a,represent:o,defaultStyle:"lowercase"})},{"../common":2,"../type":13}],17:[function(e,t,n){"use strict";function i(e){return 48<=e&&e<=57||65<=e&&e<=70||97<=e&&e<=102}function r(e){return 48<=e&&e<=55}function o(e){return 48<=e&&e<=57}function a(e){if(null===e)return!1;var t,n=e.length,a=0,s=!1;if(!n)return!1;if(t=e[a],"-"!==t&&"+"!==t||(t=e[++a]),"0"===t){if(a+1===n)return!0;if(t=e[++a],"b"===t){for(a++;a3)return!1;if("/"!==t[t.length-i.length-1])return!1}return!0}function r(e){var t=e,n=/\/([gim]*)$/.exec(e),i="";return"/"===t[0]&&(n&&(i=n[1]),t=t.slice(1,t.length-i.length-1)),new RegExp(t,i)}function o(e){var t="/"+e.source+"/";return e.global&&(t+="g"),e.multiline&&(t+="m"),e.ignoreCase&&(t+="i"),t}function a(e){return"[object RegExp]"===Object.prototype.toString.call(e)}var s=e("../../type");t.exports=new s("tag:yaml.org,2002:js/regexp",{kind:"scalar",resolve:i,construct:r,predicate:a,represent:o})},{"../../type":13}],20:[function(e,t,n){"use strict";function i(){return!0}function r(){}function o(){return""}function a(e){return"undefined"==typeof e}var s=e("../../type");t.exports=new s("tag:yaml.org,2002:js/undefined",{kind:"scalar",resolve:i,construct:r,predicate:a,represent:o})},{"../../type":13}],21:[function(e,t,n){"use strict";var i=e("../type");t.exports=new i("tag:yaml.org,2002:map",{kind:"mapping",construct:function(e){return null!==e?e:{}}})},{"../type":13}],22:[function(e,t,n){"use strict";function i(e){return"<<"===e||null===e}var r=e("../type");t.exports=new r("tag:yaml.org,2002:merge",{kind:"scalar",resolve:i})},{"../type":13}],23:[function(e,t,n){"use strict";function i(e){if(null===e)return!0;var t=e.length;return 1===t&&"~"===e||4===t&&("null"===e||"Null"===e||"NULL"===e)}function r(){return null}function o(e){return null===e}var a=e("../type");t.exports=new a("tag:yaml.org,2002:null",{kind:"scalar",resolve:i,construct:r,predicate:o,represent:{canonical:function(){return"~"},lowercase:function(){return"null"},uppercase:function(){return"NULL"},camelcase:function(){return"Null"}},defaultStyle:"lowercase"})},{"../type":13}],24:[function(e,t,n){"use strict";function i(e){if(null===e)return!0;var t,n,i,r,o,c=[],u=e;for(t=0,n=u.length;t Date: Fri, 10 Feb 2017 12:41:37 +0900 Subject: [PATCH 25/29] STORM-2344: CHANGELOG --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 272ffb189a6..42b20004833 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -198,6 +198,7 @@ * STORM-1769: Added a test to check local nimbus with notifier plugin ## 1.1.0 + * STORM-2344: Flux YAML File Viewer for Nimbus UI * STORM-2281: Running Multiple Kafka Spouts (Trident) Throws Illegal State Exception * STORM-2296: Kafka spout no dup on leader changes * STORM-2014: New Kafka spout duplicates checking if failed messages have reached max retries From 914a4768c9a8aa0320b49ff4bfb1ec338bf1d042 Mon Sep 17 00:00:00 2001 From: mingmxu Date: Fri, 3 Feb 2017 12:03:37 -0800 Subject: [PATCH 26/29] STORM-2340 fix AutoCommitMode issue in KafkaSpout * Closes #1919 * fix: KafkaSpout is blocked in AutoCommitMode * add comments for impacts of AutoCommitMode * add doc about how to use KafkaSpout with at-most-once. * remove at-most-once for better describe the changes; emit null msgId when AutoCommitMode; * update sample code in storm-kafka-client to use inline setProp() --- docs/storm-kafka-client.md | 22 ++++++++++ .../apache/storm/kafka/spout/KafkaSpout.java | 43 ++++++++++++------- 2 files changed, 50 insertions(+), 15 deletions(-) diff --git a/docs/storm-kafka-client.md b/docs/storm-kafka-client.md index ec5056fde5a..79c4115f784 100644 --- a/docs/storm-kafka-client.md +++ b/docs/storm-kafka-client.md @@ -345,3 +345,25 @@ Currently the Kafka spout has has the following default values, which have shown * offset.commit.period.ms = 30000 (30s) * max.uncommitted.offsets = 10000000
+ +# Kafka AutoCommitMode + +If reliability isn't important to you -- that is, you don't care about losing tuples in failure situations --, and want to remove the overhead of tuple tracking, then you can run a KafkaSpout with AutoCommitMode. + +To enable it, you need to: +* set Config.TOPOLOGY_ACKERS to 0; +* enable *AutoCommitMode* in Kafka consumer configuration; + +Here's one example to set AutoCommitMode in KafkaSpout: +```java +KafkaSpoutConfig kafkaConf = KafkaSpoutConfig + .builder(String bootstrapServers, String ... topics) + .setProp(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true") + .setFirstPollOffsetStrategy(FirstPollOffsetStrategy.EARLIEST) + .build(); +``` + +*Note that it's not exactly At-Most-Once in Storm, as offset is committed periodically by Kafka consumer, some tuples could be replayed when KafkaSpout is crashed.* + + + diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java index 864235c2c34..b96f3f9e1a5 100644 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java @@ -78,10 +78,10 @@ public class KafkaSpout extends BaseRichSpout { private transient boolean initialized; // Flag indicating that the spout is still undergoing initialization process. // Initialization is only complete after the first call to KafkaSpoutConsumerRebalanceListener.onPartitionsAssigned() - transient Map acked; // Tuples that were successfully acked. These tuples will be committed periodically when the commit timer expires, after consumer rebalance, or on close/deactivate - private transient Set emitted; // Tuples that have been emitted but that are "on the wire", i.e. pending being acked or failed + transient Map acked; // Tuples that were successfully acked. These tuples will be committed periodically when the commit timer expires, after consumer rebalance, or on close/deactivate. Not used if it's AutoCommitMode + private transient Set emitted; // Tuples that have been emitted but that are "on the wire", i.e. pending being acked or failed. Not used if it's AutoCommitMode private transient Iterator> waitingToEmit; // Records that have been polled and are queued to be emitted in the nextTuple() call. One record is emitted per nextTuple() - private transient long numUncommittedOffsets; // Number of offsets that have been polled and emitted but not yet been committed + private transient long numUncommittedOffsets; // Number of offsets that have been polled and emitted but not yet been committed. Not used if it's AutoCommitMode private transient TopologyContext context; private transient Timer refreshSubscriptionTimer; // Used to say when a subscription should be refreshed @@ -107,7 +107,8 @@ public void open(Map conf, TopologyContext context, SpoutOutputCollector collect // Offset management firstPollOffsetStrategy = kafkaSpoutConfig.getFirstPollOffsetStrategy(); - consumerAutoCommitMode = kafkaSpoutConfig.isConsumerAutoCommitMode(); + // with AutoCommitMode, offset will be periodically committed in the background by Kafka consumer + consumerAutoCommitMode = kafkaSpoutConfig.isConsumerAutoCommitMode(); // Retries management retryService = kafkaSpoutConfig.getRetryService(); @@ -242,14 +243,15 @@ private boolean commit() { private boolean poll() { final int maxUncommittedOffsets = kafkaSpoutConfig.getMaxUncommittedOffsets(); - final boolean poll = !waitingToEmit() && numUncommittedOffsets < maxUncommittedOffsets; + final boolean poll = !waitingToEmit() + && ( numUncommittedOffsets < maxUncommittedOffsets || consumerAutoCommitMode); if (!poll) { if (waitingToEmit()) { LOG.debug("Not polling. Tuples waiting to be emitted. [{}] uncommitted offsets across all topic partitions", numUncommittedOffsets); } - if (numUncommittedOffsets >= maxUncommittedOffsets) { + if (numUncommittedOffsets >= maxUncommittedOffsets && !consumerAutoCommitMode) { LOG.debug("Not polling. [{}] uncommitted offsets across all topic partitions has reached the threshold of [{}]", numUncommittedOffsets, maxUncommittedOffsets); } } @@ -314,15 +316,26 @@ private boolean emitTupleIfNotEmitted(ConsumerRecord record) { boolean isScheduled = retryService.isScheduled(msgId); if (!isScheduled || retryService.isReady(msgId)) { // not scheduled <=> never failed (i.e. never emitted) or ready to be retried final List tuple = kafkaSpoutConfig.getTranslator().apply(record); - if (tuple instanceof KafkaTuple) { - collector.emit(((KafkaTuple)tuple).getStream(), tuple, msgId); - } else { - collector.emit(tuple, msgId); - } - emitted.add(msgId); - numUncommittedOffsets++; - if (isScheduled) { // Was scheduled for retry, now being re-emitted. Remove from schedule. - retryService.remove(msgId); + + if(consumerAutoCommitMode){ + if (tuple instanceof KafkaTuple) { + collector.emit(((KafkaTuple)tuple).getStream(), tuple); + } else { + collector.emit(tuple); + } + }else{ + if (tuple instanceof KafkaTuple) { + collector.emit(((KafkaTuple)tuple).getStream(), tuple, msgId); + } else { + collector.emit(tuple, msgId); + } + + emitted.add(msgId); + numUncommittedOffsets++; + + if (isScheduled) { // Was scheduled for retry, now being re-emitted. Remove from schedule. + retryService.remove(msgId); + } } LOG.trace("Emitted tuple [{}] for record [{}]", tuple, record); return true; From db4695d2dd11c564f530177a1090a6dc23db3db4 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Tue, 14 Feb 2017 11:54:45 +0900 Subject: [PATCH 27/29] STORM-2340: CHANGELOG --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 42b20004833..8f731bd81d6 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -198,6 +198,7 @@ * STORM-1769: Added a test to check local nimbus with notifier plugin ## 1.1.0 + * STORM-2340: fix AutoCommitMode issue in KafkaSpout * STORM-2344: Flux YAML File Viewer for Nimbus UI * STORM-2281: Running Multiple Kafka Spouts (Trident) Throws Illegal State Exception * STORM-2296: Kafka spout no dup on leader changes From 6e75016c45c602c874086dea26324ca413f0c141 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stig=20Rohde=20D=C3=B8ssing?= Date: Tue, 14 Feb 2017 21:31:45 +0100 Subject: [PATCH 28/29] STORM-2250: Kafka spout refactoring to increase modularity and testability. Also support nanoseconds in Storm time simulation --- external/storm-kafka-client/pom.xml | 9 +- .../apache/storm/kafka/spout/KafkaSpout.java | 159 ++------- .../kafka/spout/internal/OffsetManager.java | 157 ++++++++ .../storm/kafka/spout/internal/Timer.java | 7 +- .../spout/ByTopicRecordTranslatorTest.java | 2 +- .../spout/DefaultRecordTranslatorTest.java | 2 +- .../kafka/spout/KafkaSpoutConfigTest.java | 4 +- .../kafka/spout/KafkaSpoutRebalanceTest.java | 82 ++--- .../spout/SingleTopicKafkaSpoutTest.java | 334 ++++++++++-------- .../KafkaSpoutTopologyMainNamedTopics.java | 6 +- .../KafkaSpoutTopologyMainWildcardTopics.java | 2 +- pom.xml | 1 - .../src/jvm/org/apache/storm/utils/Time.java | 146 +++++--- 13 files changed, 524 insertions(+), 387 deletions(-) create mode 100755 external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/OffsetManager.java diff --git a/external/storm-kafka-client/pom.xml b/external/storm-kafka-client/pom.xml index 97ed35919a0..0fdb64d85e9 100644 --- a/external/storm-kafka-client/pom.xml +++ b/external/storm-kafka-client/pom.xml @@ -77,7 +77,13 @@ org.hamcrest - hamcrest-all + hamcrest-core + 1.3 + test + + + org.hamcrest + hamcrest-library 1.3 test @@ -90,7 +96,6 @@ org.slf4j log4j-over-slf4j - ${log4j-over-slf4j.version} test diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java index b96f3f9e1a5..f8a576c1161 100644 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java @@ -25,16 +25,13 @@ import java.util.Collection; import java.util.Collections; -import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.NavigableSet; import java.util.Set; -import java.util.TreeSet; import java.util.concurrent.TimeUnit; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; @@ -48,6 +45,7 @@ import org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy; import org.apache.storm.kafka.spout.internal.KafkaConsumerFactory; import org.apache.storm.kafka.spout.internal.KafkaConsumerFactoryDefault; +import org.apache.storm.kafka.spout.internal.OffsetManager; import org.apache.storm.kafka.spout.internal.Timer; import org.apache.storm.spout.SpoutOutputCollector; import org.apache.storm.task.TopologyContext; @@ -58,19 +56,19 @@ public class KafkaSpout extends BaseRichSpout { private static final long serialVersionUID = 4151921085047987154L; + //Initial delay for the commit and subscription refresh timers + public static final long TIMER_DELAY_MS = 500; private static final Logger LOG = LoggerFactory.getLogger(KafkaSpout.class); - private static final Comparator OFFSET_COMPARATOR = new OffsetComparator(); // Storm protected SpoutOutputCollector collector; // Kafka private final KafkaSpoutConfig kafkaSpoutConfig; - private final KafkaConsumerFactory kafkaConsumerFactory; + private KafkaConsumerFactory kafkaConsumerFactory; private transient KafkaConsumer kafkaConsumer; private transient boolean consumerAutoCommitMode; - // Bookkeeping private transient FirstPollOffsetStrategy firstPollOffsetStrategy; // Strategy to determine the fetch offset of the first realized by the spout upon activation private transient KafkaSpoutRetryService retryService; // Class that has the logic to handle tuple failure @@ -78,7 +76,7 @@ public class KafkaSpout extends BaseRichSpout { private transient boolean initialized; // Flag indicating that the spout is still undergoing initialization process. // Initialization is only complete after the first call to KafkaSpoutConsumerRebalanceListener.onPartitionsAssigned() - transient Map acked; // Tuples that were successfully acked. These tuples will be committed periodically when the commit timer expires, after consumer rebalance, or on close/deactivate. Not used if it's AutoCommitMode + private transient Map acked; // Tuples that were successfully acked. These tuples will be committed periodically when the commit timer expires, after consumer rebalance, or on close/deactivate private transient Set emitted; // Tuples that have been emitted but that are "on the wire", i.e. pending being acked or failed. Not used if it's AutoCommitMode private transient Iterator> waitingToEmit; // Records that have been polled and are queued to be emitted in the nextTuple() call. One record is emitted per nextTuple() private transient long numUncommittedOffsets; // Number of offsets that have been polled and emitted but not yet been committed. Not used if it's AutoCommitMode @@ -87,13 +85,13 @@ public class KafkaSpout extends BaseRichSpout { public KafkaSpout(KafkaSpoutConfig kafkaSpoutConfig) { - this(kafkaSpoutConfig, new KafkaConsumerFactoryDefault()); + this(kafkaSpoutConfig, new KafkaConsumerFactoryDefault<>()); } //This constructor is here for testing KafkaSpout(KafkaSpoutConfig kafkaSpoutConfig, KafkaConsumerFactory kafkaConsumerFactory) { - this.kafkaSpoutConfig = kafkaSpoutConfig; // Pass in configuration this.kafkaConsumerFactory = kafkaConsumerFactory; + this.kafkaSpoutConfig = kafkaSpoutConfig; } @Override @@ -114,9 +112,9 @@ public void open(Map conf, TopologyContext context, SpoutOutputCollector collect retryService = kafkaSpoutConfig.getRetryService(); if (!consumerAutoCommitMode) { // If it is auto commit, no need to commit offsets manually - commitTimer = new Timer(500, kafkaSpoutConfig.getOffsetsCommitPeriodMs(), TimeUnit.MILLISECONDS); + commitTimer = new Timer(TIMER_DELAY_MS, kafkaSpoutConfig.getOffsetsCommitPeriodMs(), TimeUnit.MILLISECONDS); } - refreshSubscriptionTimer = new Timer(500, kafkaSpoutConfig.getPartitionRefreshPeriodMs(), TimeUnit.MILLISECONDS); + refreshSubscriptionTimer = new Timer(TIMER_DELAY_MS, kafkaSpoutConfig.getPartitionRefreshPeriodMs(), TimeUnit.MILLISECONDS); acked = new HashMap<>(); emitted = new HashSet<>(); @@ -198,7 +196,7 @@ private long doSeek(TopicPartition tp, OffsetAndMetadata committedOffset) { private void setAcked(TopicPartition tp, long fetchOffset) { // If this partition was previously assigned to this spout, leave the acked offsets as they were to resume where it left off if (!consumerAutoCommitMode && !acked.containsKey(tp)) { - acked.put(tp, new OffsetEntry(tp, fetchOffset)); + acked.put(tp, new OffsetManager(tp, fetchOffset)); } } @@ -290,7 +288,7 @@ private void doSeekRetriableTopicPartitions() { if (offsetAndMeta != null) { kafkaConsumer.seek(rtp, offsetAndMeta.offset() + 1); // seek to the next offset that is ready to commit in next commit cycle } else { - kafkaConsumer.seek(rtp, acked.get(rtp).committedOffset + 1); // Seek to last committed offset + kafkaConsumer.seek(rtp, acked.get(rtp).getCommittedOffset() + 1); // Seek to last committed offset } } } @@ -347,7 +345,7 @@ private boolean emitTupleIfNotEmitted(ConsumerRecord record) { private void commitOffsetsForAckedTuples() { // Find offsets that are ready to be committed for every topic partition final Map nextCommitOffsets = new HashMap<>(); - for (Map.Entry tpOffset : acked.entrySet()) { + for (Map.Entry tpOffset : acked.entrySet()) { final OffsetAndMetadata nextCommitOffset = tpOffset.getValue().findNextCommitOffset(); if (nextCommitOffset != null) { nextCommitOffsets.put(tpOffset.getKey(), nextCommitOffset); @@ -360,9 +358,14 @@ private void commitOffsetsForAckedTuples() { LOG.debug("Offsets successfully committed to Kafka [{}]", nextCommitOffsets); // Instead of iterating again, it would be possible to commit and update the state for each TopicPartition // in the prior loop, but the multiple network calls should be more expensive than iterating twice over a small loop - for (Map.Entry tpOffset : acked.entrySet()) { - final OffsetEntry offsetEntry = tpOffset.getValue(); - offsetEntry.commit(nextCommitOffsets.get(tpOffset.getKey())); + for (Map.Entry tpOffset : nextCommitOffsets.entrySet()) { + //Update the OffsetManager for each committed partition, and update numUncommittedOffsets + final TopicPartition tp = tpOffset.getKey(); + final OffsetManager offsetManager = acked.get(tp); + long numCommittedOffsets = offsetManager.commit(tpOffset.getValue()); + numUncommittedOffsets -= numCommittedOffsets; + LOG.debug("[{}] uncommitted offsets across all topic partitions", + numUncommittedOffsets); } } else { LOG.trace("No offsets to commit. {}", this); @@ -483,127 +486,7 @@ public Map getComponentConfiguration () { private String getTopicsString() { return kafkaSpoutConfig.getSubscription().getTopicsString(); } +} - // ======= Offsets Commit Management ========== - - private static class OffsetComparator implements Comparator { - public int compare(KafkaSpoutMessageId m1, KafkaSpoutMessageId m2) { - return m1.offset() < m2.offset() ? -1 : m1.offset() == m2.offset() ? 0 : 1; - } - } - - /** - * This class is not thread safe - */ - class OffsetEntry { - private final TopicPartition tp; - private final long initialFetchOffset; /* First offset to be fetched. It is either set to the beginning, end, or to the first uncommitted offset. - * Initial value depends on offset strategy. See KafkaSpoutConsumerRebalanceListener */ - private long committedOffset; // last offset committed to Kafka. Initially it is set to fetchOffset - 1 - private final NavigableSet ackedMsgs = new TreeSet<>(OFFSET_COMPARATOR); // acked messages sorted by ascending order of offset - - public OffsetEntry(TopicPartition tp, long initialFetchOffset) { - this.tp = tp; - this.initialFetchOffset = initialFetchOffset; - this.committedOffset = initialFetchOffset - 1; - LOG.debug("Instantiated {}", this); - } - - public void add(KafkaSpoutMessageId msgId) { // O(Log N) - ackedMsgs.add(msgId); - } - - /** - * An offset is only committed when all records with lower offset have - * been acked. This guarantees that all offsets smaller than the - * committedOffset have been delivered. - * @return the next OffsetAndMetadata to commit, or null if no offset is ready to commit. - */ - public OffsetAndMetadata findNextCommitOffset() { - boolean found = false; - long currOffset; - long nextCommitOffset = committedOffset; - KafkaSpoutMessageId nextCommitMsg = null; // this is a convenience variable to make it faster to create OffsetAndMetadata - - for (KafkaSpoutMessageId currAckedMsg : ackedMsgs) { // complexity is that of a linear scan on a TreeMap - if ((currOffset = currAckedMsg.offset()) == nextCommitOffset + 1) { // found the next offset to commit - found = true; - nextCommitMsg = currAckedMsg; - nextCommitOffset = currOffset; - } else if (currAckedMsg.offset() > nextCommitOffset + 1) { // offset found is not continuous to the offsets listed to go in the next commit, so stop search - LOG.debug("topic-partition [{}] has non-continuous offset [{}]. It will be processed in a subsequent batch.", tp, currOffset); - break; - } else { - //Received a redundant ack. Ignore and continue processing. - LOG.warn("topic-partition [{}] has unexpected offset [{}]. Current committed Offset [{}]", - tp, currOffset, committedOffset); - } - } - - OffsetAndMetadata nextCommitOffsetAndMetadata = null; - if (found) { - nextCommitOffsetAndMetadata = new OffsetAndMetadata(nextCommitOffset, nextCommitMsg.getMetadata(Thread.currentThread())); - LOG.debug("topic-partition [{}] has offsets [{}-{}] ready to be committed",tp, committedOffset + 1, nextCommitOffsetAndMetadata.offset()); - } else { - LOG.debug("topic-partition [{}] has NO offsets ready to be committed", tp); - } - LOG.trace("{}", this); - return nextCommitOffsetAndMetadata; - } - - /** - * Marks an offset has committed. This method has side effects - it sets the internal state in such a way that future - * calls to {@link #findNextCommitOffset()} will return offsets greater than the offset specified, if any. - * - * @param committedOffset offset to be marked as committed - */ - public void commit(OffsetAndMetadata committedOffset) { - long numCommittedOffsets = 0; - if (committedOffset != null) { - final long oldCommittedOffset = this.committedOffset; - numCommittedOffsets = committedOffset.offset() - this.committedOffset; - this.committedOffset = committedOffset.offset(); - for (Iterator iterator = ackedMsgs.iterator(); iterator.hasNext(); ) { - if (iterator.next().offset() <= committedOffset.offset()) { - iterator.remove(); - } else { - break; - } - } - numUncommittedOffsets-= numCommittedOffsets; - LOG.debug("Committed offsets [{}-{} = {}] for topic-partition [{}]. [{}] uncommitted offsets across all topic partitions", - oldCommittedOffset + 1, this.committedOffset, numCommittedOffsets, tp, numUncommittedOffsets); - } else { - LOG.debug("Committed [{}] offsets for topic-partition [{}]. [{}] uncommitted offsets across all topic partitions", - numCommittedOffsets, tp, numUncommittedOffsets); - } - LOG.trace("{}", this); - } - - long getCommittedOffset() { - return committedOffset; - } - - public boolean isEmpty() { - return ackedMsgs.isEmpty(); - } - public boolean contains(ConsumerRecord record) { - return contains(new KafkaSpoutMessageId(record)); - } - - public boolean contains(KafkaSpoutMessageId msgId) { - return ackedMsgs.contains(msgId); - } - @Override - public String toString() { - return "OffsetEntry{" + - "topic-partition=" + tp + - ", fetchOffset=" + initialFetchOffset + - ", committedOffset=" + committedOffset + - ", ackedMsgs=" + ackedMsgs + - '}'; - } - } -} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/OffsetManager.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/OffsetManager.java new file mode 100755 index 00000000000..4ce04718f27 --- /dev/null +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/OffsetManager.java @@ -0,0 +1,157 @@ +/* + * Copyright 2016 The Apache Software Foundation. + * + * Licensed 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.storm.kafka.spout.internal; + +import java.util.Comparator; +import java.util.Iterator; +import java.util.NavigableSet; +import java.util.TreeSet; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; +import org.apache.storm.kafka.spout.KafkaSpoutMessageId; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Manages acked and committed offsets for a TopicPartition. This class is not thread safe + */ +public class OffsetManager { + + private static final Comparator OFFSET_COMPARATOR = new OffsetComparator(); + private static final Logger LOG = LoggerFactory.getLogger(OffsetManager.class); + private final TopicPartition tp; + /* First offset to be fetched. It is either set to the beginning, end, or to the first uncommitted offset. + * Initial value depends on offset strategy. See KafkaSpoutConsumerRebalanceListener */ + private final long initialFetchOffset; + // Last offset committed to Kafka. Initially it is set to fetchOffset - 1 + private long committedOffset; + // Acked messages sorted by ascending order of offset + private final NavigableSet ackedMsgs = new TreeSet<>(OFFSET_COMPARATOR); + + public OffsetManager(TopicPartition tp, long initialFetchOffset) { + this.tp = tp; + this.initialFetchOffset = initialFetchOffset; + this.committedOffset = initialFetchOffset - 1; + LOG.debug("Instantiated {}", this); + } + + public void add(KafkaSpoutMessageId msgId) { // O(Log N) + ackedMsgs.add(msgId); + } + + /** + * An offset is only committed when all records with lower offset have been + * acked. This guarantees that all offsets smaller than the committedOffset + * have been delivered. + * + * @return the next OffsetAndMetadata to commit, or null if no offset is + * ready to commit. + */ + public OffsetAndMetadata findNextCommitOffset() { + boolean found = false; + long currOffset; + long nextCommitOffset = committedOffset; + KafkaSpoutMessageId nextCommitMsg = null; // this is a convenience variable to make it faster to create OffsetAndMetadata + + for (KafkaSpoutMessageId currAckedMsg : ackedMsgs) { // complexity is that of a linear scan on a TreeMap + if ((currOffset = currAckedMsg.offset()) == nextCommitOffset + 1) { // found the next offset to commit + found = true; + nextCommitMsg = currAckedMsg; + nextCommitOffset = currOffset; + } else if (currAckedMsg.offset() > nextCommitOffset + 1) { // offset found is not continuous to the offsets listed to go in the next commit, so stop search + LOG.debug("topic-partition [{}] has non-continuous offset [{}]. It will be processed in a subsequent batch.", tp, currOffset); + break; + } else { + //Received a redundant ack. Ignore and continue processing. + LOG.warn("topic-partition [{}] has unexpected offset [{}]. Current committed Offset [{}]", + tp, currOffset, committedOffset); + } + } + + OffsetAndMetadata nextCommitOffsetAndMetadata = null; + if (found) { + nextCommitOffsetAndMetadata = new OffsetAndMetadata(nextCommitOffset, nextCommitMsg.getMetadata(Thread.currentThread())); + LOG.debug("topic-partition [{}] has offsets [{}-{}] ready to be committed", tp, committedOffset + 1, nextCommitOffsetAndMetadata.offset()); + } else { + LOG.debug("topic-partition [{}] has NO offsets ready to be committed", tp); + } + LOG.trace("{}", this); + return nextCommitOffsetAndMetadata; + } + + /** + * Marks an offset has committed. This method has side effects - it sets the + * internal state in such a way that future calls to + * {@link #findNextCommitOffset()} will return offsets greater than the + * offset specified, if any. + * + * @param committedOffset offset to be marked as committed + * @return Number of offsets committed in this commit + */ + public long commit(OffsetAndMetadata committedOffset) { + long preCommitCommittedOffsets = this.committedOffset; + long numCommittedOffsets = committedOffset.offset() - this.committedOffset; + this.committedOffset = committedOffset.offset(); + for (Iterator iterator = ackedMsgs.iterator(); iterator.hasNext();) { + if (iterator.next().offset() <= committedOffset.offset()) { + iterator.remove(); + } else { + break; + } + } + LOG.trace("{}", this); + + LOG.debug("Committed offsets [{}-{} = {}] for topic-partition [{}].", + preCommitCommittedOffsets + 1, this.committedOffset, numCommittedOffsets, tp); + + return numCommittedOffsets; + } + + public long getCommittedOffset() { + return committedOffset; + } + + public boolean isEmpty() { + return ackedMsgs.isEmpty(); + } + + public boolean contains(ConsumerRecord record) { + return contains(new KafkaSpoutMessageId(record)); + } + + public boolean contains(KafkaSpoutMessageId msgId) { + return ackedMsgs.contains(msgId); + } + + @Override + public String toString() { + return "OffsetManager{" + + "topic-partition=" + tp + + ", fetchOffset=" + initialFetchOffset + + ", committedOffset=" + committedOffset + + ", ackedMsgs=" + ackedMsgs + + '}'; + } + + private static class OffsetComparator implements Comparator { + + @Override + public int compare(KafkaSpoutMessageId m1, KafkaSpoutMessageId m2) { + return m1.offset() < m2.offset() ? -1 : m1.offset() == m2.offset() ? 0 : 1; + } + } +} diff --git a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/Timer.java b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/Timer.java index d51104df82f..2a2e1cb69c1 100644 --- a/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/Timer.java +++ b/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/internal/Timer.java @@ -18,6 +18,7 @@ package org.apache.storm.kafka.spout.internal; import java.util.concurrent.TimeUnit; +import org.apache.storm.utils.Time; public class Timer { private final long delay; @@ -41,7 +42,7 @@ public Timer(long delay, long period, TimeUnit timeUnit) { this.timeUnit = timeUnit; periodNanos = timeUnit.toNanos(period); - start = System.nanoTime() + timeUnit.toNanos(delay); + start = Time.nanoTime() + timeUnit.toNanos(delay); } public long period() { @@ -65,9 +66,9 @@ public TimeUnit getTimeUnit() { * otherwise. */ public boolean isExpiredResetOnTrue() { - final boolean expired = System.nanoTime() - start > periodNanos; + final boolean expired = Time.nanoTime() - start >= periodNanos; if (expired) { - start = System.nanoTime(); + start = Time.nanoTime(); } return expired; } diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/ByTopicRecordTranslatorTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/ByTopicRecordTranslatorTest.java index fd53b1504c2..1e4b43b6dd4 100644 --- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/ByTopicRecordTranslatorTest.java +++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/ByTopicRecordTranslatorTest.java @@ -17,7 +17,7 @@ */ package org.apache.storm.kafka.spout; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; import java.util.Arrays; import java.util.HashSet; diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/DefaultRecordTranslatorTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/DefaultRecordTranslatorTest.java index f4275e49d10..681953d1f77 100644 --- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/DefaultRecordTranslatorTest.java +++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/DefaultRecordTranslatorTest.java @@ -17,7 +17,7 @@ */ package org.apache.storm.kafka.spout; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; import java.util.Arrays; diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutConfigTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutConfigTest.java index 08220dd207e..57e01205d69 100644 --- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutConfigTest.java +++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutConfigTest.java @@ -17,7 +17,9 @@ */ package org.apache.storm.kafka.spout; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; import java.util.HashMap; diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutRebalanceTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutRebalanceTest.java index 68fd4a6fe8a..b882b6759d3 100644 --- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutRebalanceTest.java +++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/KafkaSpoutRebalanceTest.java @@ -25,6 +25,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.reset; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -44,6 +45,8 @@ import org.apache.storm.kafka.spout.internal.KafkaConsumerFactory; import org.apache.storm.spout.SpoutOutputCollector; import org.apache.storm.task.TopologyContext; +import org.apache.storm.utils.Time; +import org.apache.storm.utils.Time.SimulatedTime; import org.junit.Before; import org.junit.Test; import org.mockito.ArgumentCaptor; @@ -55,20 +58,18 @@ public class KafkaSpoutRebalanceTest { @Captor private ArgumentCaptor> commitCapture; - private TopologyContext contextMock; - private SpoutOutputCollector collectorMock; - private Map conf; + private final long offsetCommitPeriodMs = 2_000; + private final TopologyContext contextMock = mock(TopologyContext.class); + private final SpoutOutputCollector collectorMock = mock(SpoutOutputCollector.class); + private final Map conf = new HashMap<>(); private KafkaConsumer consumerMock; - private KafkaConsumerFactory consumerFactoryMock; + private KafkaConsumerFactory consumerFactory; @Before public void setUp() { MockitoAnnotations.initMocks(this); - contextMock = mock(TopologyContext.class); - collectorMock = mock(SpoutOutputCollector.class); - conf = new HashMap<>(); consumerMock = mock(KafkaConsumer.class); - consumerFactoryMock = (kafkaSpoutConfig) -> consumerMock; + consumerFactory = (kafkaSpoutConfig) -> consumerMock; } //Returns messageIds in order of emission @@ -93,9 +94,9 @@ private List emitOneMessagePerPartitionThenRevokeOnePartiti Map>> secondPartitionRecords = new HashMap<>(); secondPartitionRecords.put(assignedPartition, Collections.singletonList(new ConsumerRecord(assignedPartition.topic(), assignedPartition.partition(), 0L, "key", "value"))); when(consumerMock.poll(anyLong())) - .thenReturn(new ConsumerRecords(firstPartitionRecords)) - .thenReturn(new ConsumerRecords(secondPartitionRecords)) - .thenReturn(new ConsumerRecords(Collections.emptyMap())); + .thenReturn(new ConsumerRecords(firstPartitionRecords)) + .thenReturn(new ConsumerRecords(secondPartitionRecords)) + .thenReturn(new ConsumerRecords(Collections.emptyMap())); //Emit the messages spout.nextTuple(); @@ -109,7 +110,7 @@ private List emitOneMessagePerPartitionThenRevokeOnePartiti //Now rebalance consumerRebalanceListener.onPartitionsRevoked(assignedPartitions); consumerRebalanceListener.onPartitionsAssigned(Collections.singleton(assignedPartition)); - + List emittedMessageIds = new ArrayList<>(); emittedMessageIds.add(messageIdForRevokedPartition.getValue()); emittedMessageIds.add(messageIdForAssignedPartition.getValue()); @@ -119,47 +120,48 @@ private List emitOneMessagePerPartitionThenRevokeOnePartiti @Test public void spoutMustIgnoreAcksForTuplesItIsNotAssignedAfterRebalance() throws Exception { //Acking tuples for partitions that are no longer assigned is useless since the spout will not be allowed to commit them - KafkaSpout spout = new KafkaSpout<>(getKafkaSpoutConfig(-1, 10), consumerFactoryMock); - String topic = SingleTopicKafkaSpoutConfiguration.TOPIC; - TopicPartition partitionThatWillBeRevoked = new TopicPartition(topic, 1); - TopicPartition assignedPartition = new TopicPartition(topic, 2); - - //Emit a message on each partition and revoke the first partition - List emittedMessageIds = emitOneMessagePerPartitionThenRevokeOnePartition(spout, partitionThatWillBeRevoked, assignedPartition); - - //Ack both emitted tuples - spout.ack(emittedMessageIds.get(0)); - spout.ack(emittedMessageIds.get(1)); - - //Ensure the commit timer has expired - Thread.sleep(510); - - //Make the spout commit any acked tuples - spout.nextTuple(); - //Verify that it only committed the message on the assigned partition - verify(consumerMock).commitSync(commitCapture.capture()); - - Map commitCaptureMap = commitCapture.getValue(); - assertThat(commitCaptureMap, hasKey(assignedPartition)); - assertThat(commitCaptureMap, not(hasKey(partitionThatWillBeRevoked))); + try (SimulatedTime simulatedTime = new SimulatedTime()) { + KafkaSpout spout = new KafkaSpout<>(getKafkaSpoutConfig(-1, this.offsetCommitPeriodMs), consumerFactory); + String topic = SingleTopicKafkaSpoutConfiguration.TOPIC; + TopicPartition partitionThatWillBeRevoked = new TopicPartition(topic, 1); + TopicPartition assignedPartition = new TopicPartition(topic, 2); + + //Emit a message on each partition and revoke the first partition + List emittedMessageIds = emitOneMessagePerPartitionThenRevokeOnePartition(spout, partitionThatWillBeRevoked, assignedPartition); + + //Ack both emitted tuples + spout.ack(emittedMessageIds.get(0)); + spout.ack(emittedMessageIds.get(1)); + + //Ensure the commit timer has expired + Time.advanceTime(offsetCommitPeriodMs + KafkaSpout.TIMER_DELAY_MS); + //Make the spout commit any acked tuples + spout.nextTuple(); + //Verify that it only committed the message on the assigned partition + verify(consumerMock, times(1)).commitSync(commitCapture.capture()); + + Map commitCaptureMap = commitCapture.getValue(); + assertThat(commitCaptureMap, hasKey(assignedPartition)); + assertThat(commitCaptureMap, not(hasKey(partitionThatWillBeRevoked))); + } } - + @Test public void spoutMustIgnoreFailsForTuplesItIsNotAssignedAfterRebalance() throws Exception { //Failing tuples for partitions that are no longer assigned is useless since the spout will not be allowed to commit them if they later pass KafkaSpoutRetryService retryServiceMock = mock(KafkaSpoutRetryService.class); - KafkaSpout spout = new KafkaSpout<>(getKafkaSpoutConfig(-1, 10, retryServiceMock), consumerFactoryMock); + KafkaSpout spout = new KafkaSpout<>(getKafkaSpoutConfig(-1, 10, retryServiceMock), consumerFactory); String topic = SingleTopicKafkaSpoutConfiguration.TOPIC; TopicPartition partitionThatWillBeRevoked = new TopicPartition(topic, 1); TopicPartition assignedPartition = new TopicPartition(topic, 2); - + //Emit a message on each partition and revoke the first partition List emittedMessageIds = emitOneMessagePerPartitionThenRevokeOnePartition(spout, partitionThatWillBeRevoked, assignedPartition); - + //Fail both emitted tuples spout.fail(emittedMessageIds.get(0)); spout.fail(emittedMessageIds.get(1)); - + //Check that only the tuple on the currently assigned partition is retried verify(retryServiceMock, never()).schedule(emittedMessageIds.get(0)); verify(retryServiceMock).schedule(emittedMessageIds.get(1)); diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/SingleTopicKafkaSpoutTest.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/SingleTopicKafkaSpoutTest.java index c5e4e31a43d..fdc97347b60 100644 --- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/SingleTopicKafkaSpoutTest.java +++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/SingleTopicKafkaSpoutTest.java @@ -17,6 +17,8 @@ */ package org.apache.storm.kafka.spout; +import static org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.getKafkaSpoutConfig; + import info.batey.kafka.unit.KafkaUnitRule; import kafka.producer.KeyedMessage; import org.apache.kafka.clients.consumer.OffsetAndMetadata; @@ -28,21 +30,39 @@ import org.junit.Test; import org.mockito.ArgumentCaptor; -import static org.junit.Assert.*; - import java.util.Map; import java.util.stream.IntStream; -import static org.mockito.Mockito.*; -import static org.apache.storm.kafka.spout.builders.SingleTopicKafkaSpoutConfiguration.*; + +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.Matchers.anyObject; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.reset; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import java.util.HashMap; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.TopicPartition; +import org.apache.storm.kafka.spout.internal.KafkaConsumerFactory; +import org.apache.storm.kafka.spout.internal.KafkaConsumerFactoryDefault; +import org.apache.storm.utils.Time; +import org.apache.storm.utils.Time.SimulatedTime; +import org.junit.Before; +import org.mockito.Captor; +import org.mockito.MockitoAnnotations; public class SingleTopicKafkaSpoutTest { private class SpoutContext { + public KafkaSpout spout; public SpoutOutputCollector collector; public SpoutContext(KafkaSpout spout, - SpoutOutputCollector collector) { + SpoutOutputCollector collector) { this.spout = spout; this.collector = collector; } @@ -51,190 +71,206 @@ public SpoutContext(KafkaSpout spout, @Rule public KafkaUnitRule kafkaUnitRule = new KafkaUnitRule(); - void populateTopicData(String topicName, int msgCount) { + @Captor + private ArgumentCaptor> commitCapture; + + private final TopologyContext topologyContext = mock(TopologyContext.class); + private final Map conf = new HashMap<>(); + private final SpoutOutputCollector collector = mock(SpoutOutputCollector.class); + private final long commitOffsetPeriodMs = 2_000; + private KafkaConsumer consumerSpy; + private KafkaConsumerFactory consumerFactory; + private KafkaSpout spout; + + @Before + public void setUp() { + MockitoAnnotations.initMocks(this); + KafkaSpoutConfig spoutConfig = getKafkaSpoutConfig(kafkaUnitRule.getKafkaPort(), commitOffsetPeriodMs); + this.consumerSpy = spy(new KafkaConsumerFactoryDefault().createConsumer(spoutConfig)); + this.consumerFactory = (kafkaSpoutConfig) -> consumerSpy; + this.spout = new KafkaSpout<>(spoutConfig, consumerFactory); + } + + private void populateTopicData(String topicName, int msgCount) { kafkaUnitRule.getKafkaUnit().createTopic(topicName); IntStream.range(0, msgCount).forEach(value -> { KeyedMessage keyedMessage = new KeyedMessage<>( - topicName, Integer.toString(value), - Integer.toString(value)); + topicName, Integer.toString(value), + Integer.toString(value)); kafkaUnitRule.getKafkaUnit().sendMessages(keyedMessage); }); } - SpoutContext initializeSpout(int msgCount) { + private void initializeSpout(int msgCount) { populateTopicData(SingleTopicKafkaSpoutConfiguration.TOPIC, msgCount); - int kafkaPort = kafkaUnitRule.getKafkaPort(); - - TopologyContext topology = mock(TopologyContext.class); - SpoutOutputCollector collector = mock(SpoutOutputCollector.class); - Map conf = mock(Map.class); - - KafkaSpout spout = new KafkaSpout<>(getKafkaSpoutConfig(kafkaPort)); - spout.open(conf, topology, collector); + spout.open(conf, topologyContext, collector); spout.activate(); - return new SpoutContext(spout, collector); } + /* - * Asserts that the next possible offset to commit or the committed offset is the provided offset. - * An offset that is ready to be committed is not guarenteed to be already committed. + * Asserts that commitSync has been called once, + * that there are only commits on one topic, + * and that the committed offset covers messageCount messages */ - private void assertOffsetCommitted(int offset, KafkaSpout.OffsetEntry entry) { - - boolean currentOffsetMatch = entry.getCommittedOffset() == offset; - OffsetAndMetadata nextOffset = entry.findNextCommitOffset(); - boolean nextOffsetMatch = nextOffset != null && nextOffset.offset() == offset; - assertTrue("Next offset: " + - entry.findNextCommitOffset() + - " OR current offset: " + - entry.getCommittedOffset() + - " must equal desired offset: " + - offset, - currentOffsetMatch | nextOffsetMatch); + private void verifyAllMessagesCommitted(long messageCount) { + verify(consumerSpy, times(1)).commitSync(commitCapture.capture()); + Map commits = commitCapture.getValue(); + assertThat("Expected commits for only one topic partition", commits.entrySet().size(), is(1)); + OffsetAndMetadata offset = commits.entrySet().iterator().next().getValue(); + assertThat("Expected committed offset to cover all emitted messages", offset.offset(), is(messageCount - 1)); } @Test public void shouldContinueWithSlowDoubleAcks() throws Exception { - int messageCount = 20; - SpoutContext context = initializeSpout(messageCount); - - //play 1st tuple - ArgumentCaptor messageIdToDoubleAck = ArgumentCaptor.forClass(Object.class); - context.spout.nextTuple(); - verify(context.collector).emit(anyObject(), anyObject(), messageIdToDoubleAck.capture()); - context.spout.ack(messageIdToDoubleAck.getValue()); - - IntStream.range(0, messageCount/2).forEach(value -> { - context.spout.nextTuple(); - }); - - context.spout.ack(messageIdToDoubleAck.getValue()); - - IntStream.range(0, messageCount).forEach(value -> { - context.spout.nextTuple(); - }); - - ArgumentCaptor remainingIds = ArgumentCaptor.forClass(Object.class); - - verify(context.collector, times(messageCount)).emit( - eq(SingleTopicKafkaSpoutConfiguration.STREAM), + try (SimulatedTime simulatedTime = new SimulatedTime()) { + int messageCount = 20; + initializeSpout(messageCount); + + //play 1st tuple + ArgumentCaptor messageIdToDoubleAck = ArgumentCaptor.forClass(Object.class); + spout.nextTuple(); + verify(collector).emit(anyObject(), anyObject(), messageIdToDoubleAck.capture()); + spout.ack(messageIdToDoubleAck.getValue()); + + //Emit some more messages + IntStream.range(0, messageCount / 2).forEach(value -> { + spout.nextTuple(); + }); + + spout.ack(messageIdToDoubleAck.getValue()); + + //Emit any remaining messages + IntStream.range(0, messageCount).forEach(value -> { + spout.nextTuple(); + }); + + //Verify that all messages are emitted, ack all the messages + ArgumentCaptor messageIds = ArgumentCaptor.forClass(Object.class); + verify(collector, times(messageCount)).emit(eq(SingleTopicKafkaSpoutConfiguration.STREAM), anyObject(), - remainingIds.capture()); - remainingIds.getAllValues().iterator().forEachRemaining(context.spout::ack); + messageIds.capture()); + messageIds.getAllValues().iterator().forEachRemaining(spout::ack); - context.spout.acked.values().forEach(item -> { - assertOffsetCommitted(messageCount - 1, (KafkaSpout.OffsetEntry) item); - }); + Time.advanceTime(commitOffsetPeriodMs + KafkaSpout.TIMER_DELAY_MS); + //Commit offsets + spout.nextTuple(); + + verifyAllMessagesCommitted(messageCount); + } } @Test public void shouldEmitAllMessages() throws Exception { - int messageCount = 10; - SpoutContext context = initializeSpout(messageCount); - - - IntStream.range(0, messageCount).forEach(value -> { - context.spout.nextTuple(); - ArgumentCaptor messageId = ArgumentCaptor.forClass(Object.class); - verify(context.collector).emit( + try (SimulatedTime simulatedTime = new SimulatedTime()) { + int messageCount = 10; + initializeSpout(messageCount); + + //Emit all messages and check that they are emitted. Ack the messages too + IntStream.range(0, messageCount).forEach(value -> { + spout.nextTuple(); + ArgumentCaptor messageId = ArgumentCaptor.forClass(Object.class); + verify(collector).emit( eq(SingleTopicKafkaSpoutConfiguration.STREAM), eq(new Values(SingleTopicKafkaSpoutConfiguration.TOPIC, - Integer.toString(value), - Integer.toString(value))), - messageId.capture()); - context.spout.ack(messageId.getValue()); - reset(context.collector); - }); - - context.spout.acked.values().forEach(item -> { - assertOffsetCommitted(messageCount - 1, (KafkaSpout.OffsetEntry) item); - }); + Integer.toString(value), + Integer.toString(value))), + messageId.capture()); + spout.ack(messageId.getValue()); + reset(collector); + }); + + Time.advanceTime(commitOffsetPeriodMs + KafkaSpout.TIMER_DELAY_MS); + //Commit offsets + spout.nextTuple(); + + verifyAllMessagesCommitted(messageCount); + } } @Test public void shouldReplayInOrderFailedMessages() throws Exception { - int messageCount = 10; - SpoutContext context = initializeSpout(messageCount); - - //play and ack 1 tuple - ArgumentCaptor messageIdAcked = ArgumentCaptor.forClass(Object.class); - context.spout.nextTuple(); - verify(context.collector).emit(anyObject(), anyObject(), messageIdAcked.capture()); - context.spout.ack(messageIdAcked.getValue()); - reset(context.collector); - - //play and fail 1 tuple - ArgumentCaptor messageIdFailed = ArgumentCaptor.forClass(Object.class); - context.spout.nextTuple(); - verify(context.collector).emit(anyObject(), anyObject(), messageIdFailed.capture()); - context.spout.fail(messageIdFailed.getValue()); - reset(context.collector); - - //pause so that failed tuples will be retried - Thread.sleep(200); - - - //allow for some calls to nextTuple() to fail to emit a tuple - IntStream.range(0, messageCount + 5).forEach(value -> { - context.spout.nextTuple(); - }); - - ArgumentCaptor remainingMessageIds = ArgumentCaptor.forClass(Object.class); - - //1 message replayed, messageCount - 2 messages emitted for the first time - verify(context.collector, times(messageCount - 1)).emit( + try (SimulatedTime simulatedTime = new SimulatedTime()) { + int messageCount = 10; + initializeSpout(messageCount); + + //play and ack 1 tuple + ArgumentCaptor messageIdAcked = ArgumentCaptor.forClass(Object.class); + spout.nextTuple(); + verify(collector).emit(anyObject(), anyObject(), messageIdAcked.capture()); + spout.ack(messageIdAcked.getValue()); + reset(collector); + + //play and fail 1 tuple + ArgumentCaptor messageIdFailed = ArgumentCaptor.forClass(Object.class); + spout.nextTuple(); + verify(collector).emit(anyObject(), anyObject(), messageIdFailed.capture()); + spout.fail(messageIdFailed.getValue()); + reset(collector); + + //Emit all remaining messages. Failed tuples retry immediately with current configuration, so no need to wait. + IntStream.range(0, messageCount).forEach(value -> { + spout.nextTuple(); + }); + + ArgumentCaptor remainingMessageIds = ArgumentCaptor.forClass(Object.class); + //All messages except the first acked message should have been emitted + verify(collector, times(messageCount - 1)).emit( eq(SingleTopicKafkaSpoutConfiguration.STREAM), anyObject(), remainingMessageIds.capture()); - remainingMessageIds.getAllValues().iterator().forEachRemaining(context.spout::ack); + remainingMessageIds.getAllValues().iterator().forEachRemaining(spout::ack); - context.spout.acked.values().forEach(item -> { - assertOffsetCommitted(messageCount - 1, (KafkaSpout.OffsetEntry) item); - }); + Time.advanceTime(commitOffsetPeriodMs + KafkaSpout.TIMER_DELAY_MS); + //Commit offsets + spout.nextTuple(); + + verifyAllMessagesCommitted(messageCount); + } } @Test public void shouldReplayFirstTupleFailedOutOfOrder() throws Exception { - int messageCount = 10; - SpoutContext context = initializeSpout(messageCount); - - - //play 1st tuple - ArgumentCaptor messageIdToFail = ArgumentCaptor.forClass(Object.class); - context.spout.nextTuple(); - verify(context.collector).emit(anyObject(), anyObject(), messageIdToFail.capture()); - reset(context.collector); - - //play 2nd tuple - ArgumentCaptor messageIdToAck = ArgumentCaptor.forClass(Object.class); - context.spout.nextTuple(); - verify(context.collector).emit(anyObject(), anyObject(), messageIdToAck.capture()); - reset(context.collector); - - //ack 2nd tuple - context.spout.ack(messageIdToAck.getValue()); - //fail 1st tuple - context.spout.fail(messageIdToFail.getValue()); - - //pause so that failed tuples will be retried - Thread.sleep(200); - - //allow for some calls to nextTuple() to fail to emit a tuple - IntStream.range(0, messageCount + 5).forEach(value -> { - context.spout.nextTuple(); - }); - - ArgumentCaptor remainingIds = ArgumentCaptor.forClass(Object.class); - //1 message replayed, messageCount - 2 messages emitted for the first time - verify(context.collector, times(messageCount - 1)).emit( + try (SimulatedTime simulatedTime = new SimulatedTime()) { + int messageCount = 10; + initializeSpout(messageCount); + + //play 1st tuple + ArgumentCaptor messageIdToFail = ArgumentCaptor.forClass(Object.class); + spout.nextTuple(); + verify(collector).emit(anyObject(), anyObject(), messageIdToFail.capture()); + reset(collector); + + //play 2nd tuple + ArgumentCaptor messageIdToAck = ArgumentCaptor.forClass(Object.class); + spout.nextTuple(); + verify(collector).emit(anyObject(), anyObject(), messageIdToAck.capture()); + reset(collector); + + //ack 2nd tuple + spout.ack(messageIdToAck.getValue()); + //fail 1st tuple + spout.fail(messageIdToFail.getValue()); + + //Emit all remaining messages. Failed tuples retry immediately with current configuration, so no need to wait. + IntStream.range(0, messageCount).forEach(value -> { + spout.nextTuple(); + }); + + ArgumentCaptor remainingIds = ArgumentCaptor.forClass(Object.class); + //All messages except the first acked message should have been emitted + verify(collector, times(messageCount - 1)).emit( eq(SingleTopicKafkaSpoutConfiguration.STREAM), anyObject(), remainingIds.capture()); - remainingIds.getAllValues().iterator().forEachRemaining(context.spout::ack); + remainingIds.getAllValues().iterator().forEachRemaining(spout::ack); - context.spout.acked.values().forEach(item -> { - assertOffsetCommitted(messageCount - 1, (KafkaSpout.OffsetEntry) item); - }); + Time.advanceTime(commitOffsetPeriodMs + KafkaSpout.TIMER_DELAY_MS); + //Commit offsets + spout.nextTuple(); + + verifyAllMessagesCommitted(messageCount); + } } -} \ No newline at end of file +} diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainNamedTopics.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainNamedTopics.java index 2aeeb9571e9..e305c8afc54 100644 --- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainNamedTopics.java +++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainNamedTopics.java @@ -50,9 +50,9 @@ public static void main(String[] args) throws Exception { protected void runMain(String[] args) throws Exception { if (args.length == 0) { - submitTopologyLocalCluster(getTopolgyKafkaSpout(), getConfig()); + submitTopologyLocalCluster(getTopologyKafkaSpout(), getConfig()); } else { - submitTopologyRemoteCluster(args[0], getTopolgyKafkaSpout(), getConfig()); + submitTopologyRemoteCluster(args[0], getTopologyKafkaSpout(), getConfig()); } } @@ -82,7 +82,7 @@ protected Config getConfig() { return config; } - protected StormTopology getTopolgyKafkaSpout() { + protected StormTopology getTopologyKafkaSpout() { final TopologyBuilder tp = new TopologyBuilder(); tp.setSpout("kafka_spout", new KafkaSpout<>(getKafkaSpoutConfig()), 1); tp.setBolt("kafka_bolt", new KafkaSpoutTestBolt()) diff --git a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainWildcardTopics.java b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainWildcardTopics.java index d0376e68c6b..f811c7ab73d 100644 --- a/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainWildcardTopics.java +++ b/external/storm-kafka-client/src/test/java/org/apache/storm/kafka/spout/test/KafkaSpoutTopologyMainWildcardTopics.java @@ -37,7 +37,7 @@ public static void main(String[] args) throws Exception { new KafkaSpoutTopologyMainWildcardTopics().runMain(args); } - protected StormTopology getTopolgyKafkaSpout() { + protected StormTopology getTopologyKafkaSpout() { final TopologyBuilder tp = new TopologyBuilder(); tp.setSpout("kafka_spout", new KafkaSpout<>(getKafkaSpoutConfig()), 1); tp.setBolt("kafka_bolt", new KafkaSpoutTestBolt()).shuffleGrouping("kafka_spout", STREAM); diff --git a/pom.xml b/pom.xml index 13c4c3548cf..6d3543e7a1d 100644 --- a/pom.xml +++ b/pom.xml @@ -359,7 +359,6 @@ junit junit - ${junit.version} test diff --git a/storm-core/src/jvm/org/apache/storm/utils/Time.java b/storm-core/src/jvm/org/apache/storm/utils/Time.java index c5c6b6a7dcc..0401829ad0e 100644 --- a/storm-core/src/jvm/org/apache/storm/utils/Time.java +++ b/storm-core/src/jvm/org/apache/storm/utils/Time.java @@ -24,14 +24,18 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; - +/** + * This class implements time simulation support. When time simulation is enabled, methods on this class will use fixed time. + * When time simulation is disabled, methods will pass through to relevant java.lang.System/java.lang.Thread calls. + * Methods using units higher than nanoseconds will pass through to System.currentTimeMillis(). Methods supporting nanoseconds will pass through to System.nanoTime(). + */ public class Time { private static final Logger LOG = LoggerFactory.getLogger(Time.class); private static AtomicBoolean simulating = new AtomicBoolean(false); - private static AtomicLong autoAdvanceOnSleep = new AtomicLong(0); - private static volatile Map threadSleepTimes; + private static AtomicLong autoAdvanceNanosOnSleep = new AtomicLong(0); + private static volatile Map threadSleepTimesNanos; private static final Object sleepTimesLock = new Object(); - private static AtomicLong simulatedCurrTimeMs; + private static AtomicLong simulatedCurrTimeNanos; public static class SimulatedTime implements AutoCloseable { @@ -39,13 +43,13 @@ public SimulatedTime() { this(null); } - public SimulatedTime(Number ms) { + public SimulatedTime(Number advanceTimeMs) { synchronized(Time.sleepTimesLock) { Time.simulating.set(true); - Time.simulatedCurrTimeMs = new AtomicLong(0); - Time.threadSleepTimes = new ConcurrentHashMap<>(); - if (ms != null) { - Time.autoAdvanceOnSleep.set(ms.longValue()); + Time.simulatedCurrTimeNanos = new AtomicLong(0); + Time.threadSleepTimesNanos = new ConcurrentHashMap<>(); + if (advanceTimeMs != null) { + Time.autoAdvanceNanosOnSleep.set(millisToNanos(advanceTimeMs.longValue())); } LOG.warn("AutoCloseable Simulated Time Starting..."); } @@ -55,8 +59,8 @@ public SimulatedTime(Number ms) { public void close() { synchronized(Time.sleepTimesLock) { Time.simulating.set(false); - Time.autoAdvanceOnSleep.set(0); - Time.threadSleepTimes = null; + Time.autoAdvanceNanosOnSleep.set(0); + Time.threadSleepTimesNanos = null; LOG.warn("AutoCloseable Simulated Time Ending..."); } } @@ -66,8 +70,8 @@ public void close() { public static void startSimulating() { synchronized(Time.sleepTimesLock) { Time.simulating.set(true); - Time.simulatedCurrTimeMs = new AtomicLong(0); - Time.threadSleepTimes = new ConcurrentHashMap<>(); + Time.simulatedCurrTimeNanos = new AtomicLong(0); + Time.threadSleepTimesNanos = new ConcurrentHashMap<>(); LOG.warn("Simulated Time Starting..."); } } @@ -76,8 +80,8 @@ public static void startSimulating() { public static void stopSimulating() { synchronized(Time.sleepTimesLock) { Time.simulating.set(false); - Time.autoAdvanceOnSleep.set(0); - Time.threadSleepTimes = null; + Time.autoAdvanceNanosOnSleep.set(0); + Time.threadSleepTimesNanos = null; LOG.warn("Simulated Time Ending..."); } } @@ -88,44 +92,66 @@ public static boolean isSimulating() { public static void sleepUntil(long targetTimeMs) throws InterruptedException { if(simulating.get()) { - try { - synchronized(sleepTimesLock) { - if (threadSleepTimes == null) { + simulatedSleepUntilNanos(millisToNanos(targetTimeMs)); + } else { + long sleepTimeMs = targetTimeMs - currentTimeMillis(); + if(sleepTimeMs>0) { + Thread.sleep(sleepTimeMs); + } + } + } + + public static void sleepUntilNanos(long targetTimeNanos) throws InterruptedException { + if(simulating.get()) { + simulatedSleepUntilNanos(targetTimeNanos); + } else { + long sleepTimeNanos = targetTimeNanos-nanoTime(); + long sleepTimeMs = nanosToMillis(sleepTimeNanos); + int sleepTimeNanosSansMs = (int)(sleepTimeNanos%1_000_000); + if(sleepTimeNanos>0) { + Thread.sleep(sleepTimeMs, sleepTimeNanosSansMs); + } + } + } + + private static void simulatedSleepUntilNanos(long targetTimeNanos) throws InterruptedException { + try { + synchronized (sleepTimesLock) { + if (threadSleepTimesNanos == null) { + LOG.debug("{} is still sleeping after simulated time disabled.", Thread.currentThread(), new RuntimeException("STACK TRACE")); + throw new InterruptedException(); + } + threadSleepTimesNanos.put(Thread.currentThread(), new AtomicLong(targetTimeNanos)); + } + while (simulatedCurrTimeNanos.get() < targetTimeNanos) { + synchronized (sleepTimesLock) { + if (threadSleepTimesNanos == null) { LOG.debug("{} is still sleeping after simulated time disabled.", Thread.currentThread(), new RuntimeException("STACK TRACE")); throw new InterruptedException(); } - threadSleepTimes.put(Thread.currentThread(), new AtomicLong(targetTimeMs)); } - while(simulatedCurrTimeMs.get() < targetTimeMs) { - synchronized(sleepTimesLock) { - if (threadSleepTimes == null) { - LOG.debug("{} is still sleeping after simulated time disabled.", Thread.currentThread(), new RuntimeException("STACK TRACE")); - throw new InterruptedException(); - } - } - long autoAdvance = autoAdvanceOnSleep.get(); - if (autoAdvance > 0) { - advanceTime(autoAdvance); - } - Thread.sleep(10); + long autoAdvance = autoAdvanceNanosOnSleep.get(); + if (autoAdvance > 0) { + advanceTimeNanos(autoAdvance); } - } finally { - synchronized(sleepTimesLock) { - if (simulating.get() && threadSleepTimes != null) { - threadSleepTimes.remove(Thread.currentThread()); - } + Thread.sleep(10); + } + } finally { + synchronized (sleepTimesLock) { + if (simulating.get() && threadSleepTimesNanos != null) { + threadSleepTimesNanos.remove(Thread.currentThread()); } } - } else { - long sleepTime = targetTimeMs-currentTimeMillis(); - if(sleepTime>0) - Thread.sleep(sleepTime); } } public static void sleep(long ms) throws InterruptedException { sleepUntil(currentTimeMillis()+ms); } + + public static void sleepNanos(long nanos) throws InterruptedException { + sleepUntilNanos(nanoTime() + nanos); + } public static void sleepSecs (long secs) throws InterruptedException { if (secs > 0) { @@ -133,14 +159,30 @@ public static void sleepSecs (long secs) throws InterruptedException { } } + public static long nanoTime() { + if (simulating.get()) { + return simulatedCurrTimeNanos.get(); + } else { + return System.nanoTime(); + } + } + public static long currentTimeMillis() { if(simulating.get()) { - return simulatedCurrTimeMs.get(); + return nanosToMillis(simulatedCurrTimeNanos.get()); } else { return System.currentTimeMillis(); } } + public static long nanosToMillis(long nanos) { + return nanos/1_000_000; + } + + public static long millisToNanos(long millis) { + return millis*1_000_000; + } + public static long secsToMillis (int secs) { return 1000*(long) secs; } @@ -162,9 +204,17 @@ public static long deltaMs(long timeInMilliseconds) { } public static void advanceTime(long ms) { - if (!simulating.get()) throw new IllegalStateException("Cannot simulate time unless in simulation mode"); - if (ms < 0) throw new IllegalArgumentException("advanceTime only accepts positive time as an argument"); - long newTime = simulatedCurrTimeMs.addAndGet(ms); + advanceTimeNanos(millisToNanos(ms)); + } + + public static void advanceTimeNanos(long nanos) { + if (!simulating.get()) { + throw new IllegalStateException("Cannot simulate time unless in simulation mode"); + } + if (nanos < 0) { + throw new IllegalArgumentException("advanceTime only accepts positive time as an argument"); + } + long newTime = simulatedCurrTimeNanos.addAndGet(nanos); LOG.debug("Advanced simulated time to {}", newTime); } @@ -173,11 +223,13 @@ public static void advanceTimeSecs(long secs) { } public static boolean isThreadWaiting(Thread t) { - if(!simulating.get()) throw new IllegalStateException("Must be in simulation mode"); + if(!simulating.get()) { + throw new IllegalStateException("Must be in simulation mode"); + } AtomicLong time; synchronized(sleepTimesLock) { - time = threadSleepTimes.get(t); + time = threadSleepTimesNanos.get(t); } - return !t.isAlive() || time!=null && currentTimeMillis() < time.longValue(); + return !t.isAlive() || time!=null && nanoTime() < time.longValue(); } } From d235a0c1b49a4ead966d2288e81058495d2ada44 Mon Sep 17 00:00:00 2001 From: "P. Taylor Goetz" Date: Wed, 15 Feb 2017 16:34:13 -0500 Subject: [PATCH 29/29] add STORM-2250 to changelog --- CHANGELOG.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 8f731bd81d6..efc349094e3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ -## 2.0.0 +## 2.0.0 + * STORM-2250: Kafka Spout Refactoring to Increase Modularity and Testability * STORM-2346: Files with unapproved licenses: download-rc-directory.sh verify-release-file.sh * STORM-2350: Storm-HDFS's listFilesByModificationTime is broken * STORM-1961: Stream api for storm core use cases