├── README.md ├── Simois-conf.bib ├── pom.xml └── src └── main ├── SimoisStructure.png ├── java └── com │ └── basic │ └── core │ ├── Constraints.java │ ├── KafkaTopology.java │ ├── SchedulingTopologyBuilder.java │ ├── TopologyArgs.java │ ├── bolt │ ├── AggregatorBolt.java │ ├── CoinBolt.java │ ├── JoinBolt.java │ ├── PostProcessBolt.java │ ├── PredictorBolt.java │ ├── SchedulerBolt.java │ └── ShuffleBolt.java │ ├── inter │ ├── BinaryComparable.java │ └── DumpRemoveHandler.java │ └── util │ ├── CastUtils.java │ ├── FileReader.java │ ├── FileWriter.java │ ├── GeoHash.java │ ├── HdfsOperationUtil.java │ ├── LogHelpers.java │ ├── MyScheme.java │ ├── PredictHeavyLoadKeyUtil.java │ ├── PropertiesUtil.java │ ├── SchemaUtils.java │ ├── Stopwatch.java │ ├── StormRunner.java │ ├── SynopsisHashMap.java │ └── TimeUtils.java └── resources ├── log4j.properties └── simois.properties /README.md: -------------------------------------------------------------------------------- 1 | # Simois 2 | 3 | Simois is a scalable distributed stream join system, which supports efficient join operations in two streams with highly skewed data distribution. Simois can support the completeness of the join results, and greatly outperforms the existing stream join systems in terms of system throughput and the average processing latency. Experiment results show that Simois improves the system throughput significantly by 52% and reduces the average latency by 37%, compared to existing stream join systems. 4 | 5 | 6 | ## Introduction 7 | 8 | Recently, more and more applications raise the demand of processing and analyzing the streaming data in real-time. Stream joining is one of the most basic and important operation to extract valuable information from multiple data streams. Processing joining on data streams is much more challenging than performing traditional join operation on databases. Firstly, stream joining needs to ensure the completeness of the join results, that means each pair of tuples from two streams should be matched exactly once, while the two streams are continuous and unbounded. Secondly, stream joining always needs to be processed in-memory to avoid frequently disk I/O. However, memory is always limited but the data are continuous. Thirdly, stream join applications usually need to report results in near real-time style, thus, performing efficient stream joining requires high throughput and low latency processing systems. 9 | 10 | To support continuous stream joining, much effort has been done based on different parallel or distributed designs. Due to the limitation of memory capacity, a parallel stream join system commonly processes the stream join with a limited sliding window. Such designs actually significantly sacrifice the completeness of the results of stream join to achieve feasibility of the designs. Different from parallel systems, distributed join systems can support near full history join operations. 11 | 12 | Recently, Lin et al. propose a scalable distributed stream join system called BiStream (http://www.comp.nus.edu.sg/~linqian/bistream). BiStream depends on a novel stream joining model, which separates all the processing units for join operation into two symmetry sets and organizes them into a bipartite graph architecture. A key problem in BiStream is how to rationally partition tuples among different join units. BiStream introduces two partitioning strategies including random partitioning and hash partitioning. 13 | 14 | Random partitioning strategy partitions tuples of one stream to a random processing unit for storing, while the tuple of the other stream is broadcast to all the processing units for joining. Such a strategy in fact compares every two tuples in different streams regardless of the content of the tuples. Therefore, it incurs a large amount of unnecessary join operations, resulting in high latency and low throughput, especially for equi-joins. Hash partitioning strategy is a straightforward strategy for equi-joins. This strategy stores the tuples associated with a specific key in one stream to a specified processing unit using hashing. The tuples with the same key in the other stream will be hashed to the same unit for joining. However, such a strategy suffers from the load imbalance problem due to the skewed distribution of the keys, which is common in real world data. Thus, the processing units assigned the top heavy load become stragglers in the system and make the system inefficient. 15 | 16 | To deal with the above problems, we build Simois, a {S}calable d{I}stribute strea{M} j{OI}n {S}ystem. Simois can efficiently identify the exact keys which incur heavy workload in real-time stream joining. With the identification, the system accordingly evenly partitions the keys causing heavy-load using shuffling strategy, and partitions the rest stream data using hash-based schemes to avoid redundant join computations. 17 | 18 | ## Structure of Simois 19 | 20 | ![image](https://github.com/DStream-Storm/Simois/blob/master/src/main/SimoisStructure.png) 21 | 22 | Simois follows the join-biclique stream joining framework in BiStream. To implement the join-biclique model, we divide all the processing units into two sets: $R$-joining instances and $S$-joining instances. A $S$-joining instance stores the incoming tuples of stream $S$ based on the hash partitioning. The instance performs join operation whenever it receives a tuple of stream $R$, which has the associated key. Specifically, it matches the received $R$ tuple with all the $S$ tuples stored in this instance. $R$-joining instances are on the opposite. For simplicity, in the following, we only discuss about the $S$-joining instances and the symmetric $R$-instances work in the same way. 23 | 24 | The key design in Simois is the light-weighted potential heavy-load keys predictor and the differentiated scheduling strategy for both streams. Simois consists of two main components: 1) an independent predicting component for identifying the keys which incur the heaviest workload, and 2) a scheduling component in each dispatching instance. The predicting component is a standalone component separated from the original processing procedure, which collects the information of all the tuples and uses this information to identify the keys with heaviest workload. The scheduling component is embedded in each processing element instance, which stores the identified heavy-load keys and supports fast querying and efficiently scheduling. 25 | 26 | 27 | ## How to use? 28 | 29 | ### Environment 30 | 31 | We implement Simois atop Apache Storm (version 1.0.2 or higher), and deploy the system on a cluster. Each machine is equipped with an octa-core 2.4GHz Xeon CPU, 64.0GB RAM, and a 1000Mbps Ethernet interface card. One machine in the cluster serves as the master node to host the Storm Nimbus. The other machines run Storm supervisors. 32 | 33 | ### Initial Setting 34 | 35 | Install Apache Storm (Please refer to http://storm.apache.org/ to learn more). 36 | 37 | Install Apace Maven (Please refer to http://maven.apache.org/ to learn more). 38 | 39 | Build and package the example 40 | 41 | ```txt 42 | mvn clean package -Dmaven.test.skip=true 43 | ``` 44 | 45 | ### Configurations 46 | 47 | Configurations including Threshold_r, Threshold_l and Threshold_p in ./src/main/resources/simois.properties. 48 | 49 | ```txt 50 | Threshold_r=12 (by default) 51 | Threshold_l=24 (by default) 52 | Threshold_p=0.03 (by default) 53 | ``` 54 | 55 | ### Using Simois 56 | 57 | Import SchedulingTopologyBuilder in the application source code 58 | 59 | ```txt 60 | import com.basic.core.SchedulingTopologyBuilder; 61 | ``` 62 | 63 | Build SchedulingTopologyBuilder before the building of the topology 64 | 65 | ```txt 66 | SchedulingTopologyBuilder builder=new SchedulingTopologyBuilder(); 67 | ``` 68 | 69 | Generate SchedulingTopologyBuilder according to the Threshold_r, Threshold_l and Threshold_p (config in ./src/main/resources/dstream.properties). 70 | 71 | ```java 72 | SchedulingTopologyBuilder builder=new SchedulingTopologyBuilder(); 73 | ``` 74 | 75 | Set join-biclique and heavy-load key predictor in the application topology (see ./src/main/java/com/basic/core/KafkaTopology.java 76 | 77 | ```java 78 | builder.setSpout(KAFKA_SPOUT_ID_R, new KafkaSpout<>(getKafkaSpoutConfig(KAFKA_BROKER, "didiOrder" + _args.dataSize, _args.groupid)), _args.numKafkaSpouts); 79 | builder.setSpout(KAFKA_SPOUT_ID_S, new KafkaSpout<>(getKafkaSpoutConfig(KAFKA_BROKER, "didiGps" + _args.dataSize, _args.groupid)), _args.numKafkaSpouts); 80 | builder.setBolt(SHUFFLE_BOLT_ID, new ShuffleBolt(_args.dataSize), _args.numShufflers) 81 | .shuffleGrouping(KAFKA_SPOUT_ID_R) 82 | .shuffleGrouping(KAFKA_SPOUT_ID_S); 83 | 84 | builder.setDifferentiatedScheduling(SHUFFLE_BOLT_ID, Constraints.relFileds, Constraints.wordFileds); 85 | 86 | builder.setBolt(JOINER_R_BOLT_ID, joinerR, _args.numPartitionsR) 87 | .fieldsGrouping(SCHEDULER_BOLT_ID+builder.getSchedulingNum(), Constraints.nohotRFileds, new Fields(Constraints.wordFileds)) 88 | .fieldsGrouping(SCHEDULER_BOLT_ID+builder.getSchedulingNum(), Constraints.nohotSFileds, new Fields(Constraints.wordFileds)) 89 | .shuffleGrouping(SCHEDULER_BOLT_ID+builder.getSchedulingNum(), Constraints.hotRFileds) 90 | .allGrouping(SCHEDULER_BOLT_ID+builder.getSchedulingNum(), Constraints.hotSFileds); 91 | 92 | builder.setBolt(JOINER_S_BOLT_ID, joinerS, _args.numPartitionsS) 93 | .fieldsGrouping(SCHEDULER_BOLT_ID+builder.getSchedulingNum(), Constraints.nohotSFileds, new Fields(Constraints.wordFileds)) 94 | .fieldsGrouping(SCHEDULER_BOLT_ID+builder.getSchedulingNum(), Constraints.nohotRFileds, new Fields(Constraints.wordFileds)) 95 | .shuffleGrouping(SCHEDULER_BOLT_ID+builder.getSchedulingNum(), Constraints.hotSFileds) 96 | .allGrouping(SCHEDULER_BOLT_ID+builder.getSchedulingNum(), Constraints.hotRFileds); 97 | ``` 98 | 99 | ## Publications 100 | 101 | If you want to know more detailed information, please refer to this paper: 102 | 103 | Fan Zhang, Hanhua Chen, Hai Jin. "Simois: A Scalable Distributed Stream Join System with Skewed Workloads." in Proceedings of IEEE 39th International Conference on Distributed Computing Systems (ICDCS), 2019. ([Bibtex](https://github.com/CGCL-codes/Simois/blob/master/Simois-conf.bib)) 104 | 105 | 106 | ## Authors and Copyright 107 | 108 | Simois is developed in National Engineering Research Center for Big Data Technology and System, Cluster and Grid Computing Lab, Services Computing Technology and System Lab, School of Computer Science and Technology, Huazhong University of Science and Technology, Wuhan, China by Fan Zhang(zhangf@hust.edu.cn), Hanhua Chen (chen@hust.edu.cn), Hai Jin (hjin@hust.edu.cn). 109 | 110 | Copyright (C) 2019, [STCS & CGCL](http://grid.hust.edu.cn/) and [Huazhong University of Science and Technology](http://www.hust.edu.cn). 111 | 112 | 113 | -------------------------------------------------------------------------------- /Simois-conf.bib: -------------------------------------------------------------------------------- 1 | @inproceedings{Simois-conf19, 2 | author = {Fan Zhang and 3 | Hanhua Chen and 4 | Hai Jin}, 5 | title = {Simois: {A} Scalable Distributed Stream Join System with Skewed Workloads}, 6 | booktitle = {39th {IEEE} International Conference on Distributed Computing Systems, 7 | {ICDCS} 2019, Dallas, TX, USA, July 7-10, 2019}, 8 | pages = {176--185}, 9 | year = {2019} 10 | } -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/CGCL-codes/Simois/66cfeda923d077cb7d7162b5de0126f52417c756/pom.xml -------------------------------------------------------------------------------- /src/main/SimoisStructure.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/CGCL-codes/Simois/66cfeda923d077cb7d7162b5de0126f52417c756/src/main/SimoisStructure.png -------------------------------------------------------------------------------- /src/main/java/com/basic/core/Constraints.java: -------------------------------------------------------------------------------- 1 | package com.basic.core; 2 | 3 | 4 | import com.basic.core.util.PropertiesUtil; 5 | 6 | /** 7 | * locate com.basic.storm 8 | * Created by windy on 2019/1/2. 9 | */ 10 | public class Constraints { 11 | public static int Threshold_r = Integer.valueOf(PropertiesUtil.getProperties("Threshold_r"));// 12 | public static int Threshold_l = Integer.valueOf(PropertiesUtil.getProperties("Threshold_l"));// 13 | public static double Threshold_p = Double.valueOf(PropertiesUtil.getProperties("Threshold_p"));//Attenuation probability 14 | 15 | public static final String SPLITTER_BOLT_ID = "splitter-bolt"; 16 | public static final String COIN_BOLT_ID= "coin-bolt"; 17 | public static final String PREDICTOR_BOLT_ID= "predictor-bolt"; 18 | 19 | public static final String coinFileds="coin"; 20 | public static final String wordFileds="key"; 21 | public static final String relFileds="relation"; 22 | public static final String tsFileds="timestamp"; 23 | public static final String valueFileds="value"; 24 | public static final String baseCountFileds="basecount"; 25 | public static final String hotRFileds="hotR"; 26 | public static final String nohotRFileds="nohotR"; 27 | public static final String hotRFileds="hotS"; 28 | public static final String nohotRFileds="nohotS"; 29 | public static final String coinCountFileds="coincount"; 30 | public static final String typeFileds="type"; 31 | } 32 | -------------------------------------------------------------------------------- /src/main/java/com/basic/core/KafkaTopology.java: -------------------------------------------------------------------------------- 1 | package com.basic.core; 2 | 3 | import com.basic.core.Constraints; 4 | import com.basic.core.SchedulingTopologyBuilder; 5 | import com.basic.core.util.MyScheme; 6 | import org.apache.storm.Config; 7 | import org.apache.storm.LocalCluster; 8 | import org.apache.storm.StormSubmitter; 9 | import org.apache.storm.generated.AlreadyAliveException; 10 | import org.apache.storm.generated.AuthorizationException; 11 | import org.apache.storm.generated.InvalidTopologyException; 12 | import org.apache.storm.kafka.BrokerHosts; 13 | import org.apache.storm.kafka.KafkaSpout; 14 | import org.apache.storm.kafka.SpoutConfig; 15 | import org.apache.storm.kafka.ZkHosts; 16 | import org.apache.storm.spout.SchemeAsMultiScheme; 17 | import org.apache.storm.tuple.Fields; 18 | import org.apache.storm.utils.Utils; 19 | import com.esotericsoftware.minlog.Log; 20 | import org.apache.kafka.clients.consumer.ConsumerConfig; 21 | import org.apache.storm.generated.StormTopology; 22 | import org.apache.storm.kafka.spout.*; 23 | import org.apache.storm.tuple.Fields; 24 | import org.apache.storm.tuple.Values; 25 | import org.slf4j.Logger; 26 | 27 | import static org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.EARLIEST; 28 | import static org.slf4j.LoggerFactory.getLogger; 29 | 30 | import org.apache.storm.Config; 31 | import org.apache.storm.topology.TopologyBuilder; 32 | import com.basic.core.bolt.*; 33 | import com.basic.core.core.ContRandGrouping; 34 | import com.basic.core.core.FastJoinGrouping; 35 | import com.basic.core.util.FileWriter; 36 | 37 | import java.util.Arrays; 38 | import java.util.HashMap; 39 | 40 | import static com.basic.core.util.LogHelpers.logTopology; 41 | import static com.basic.core.util.StormRunner.runInCluster; 42 | import static com.basic.core.util.StormRunner.runLocally; 43 | 44 | import static com.basic.core.Constraints.SCHEDULER_BOLT_ID; 45 | 46 | public class KafkaTopology 47 | { 48 | private static final Logger LOG = getLogger(KafkaTopology.class); 49 | private static final String SHUFFLE_BOLT_ID = "shuffler"; 50 | private static final String RESHUFFLE_BOLT_ID = "reshuffler"; 51 | private static final String JOINER_R_BOLT_ID = "joiner-r"; 52 | private static final String JOINER_S_BOLT_ID = "joiner-s"; 53 | private static final String POST_PROCESS_BOLT_ID = "gatherer"; 54 | private static final String AGGREGATE_BOLT_ID = "aggregator"; 55 | 56 | public static final String KAFKA_SPOUT_ID_R ="kafka-spout-r"; 57 | public static final String KAFKA_SPOUT_ID_S ="kafka-spout-s"; 58 | //public static final String KAFKA_TEST_BOLT_ID = "kafka-test"; 59 | // public static final String KAFKA_BROKER = "node24:9092,node25:9092,node26:9092,node27:9092,node28:9092"; 60 | public static final String KAFKA_BROKER = "node95:9092,node96:9092,node97:9092,node98:9092,node99:9092"; 61 | 62 | 63 | 64 | private final TopologyArgs _args = new TopologyArgs("KafkaTopology"); 65 | 66 | public int run(String[] args) throws Exception { 67 | if (!_args.processArgs(args)) 68 | return -1; 69 | if (_args.help) 70 | return 0; 71 | else 72 | // _args.logArgs(); 73 | writeSettingsToFile(); 74 | 75 | /* build topology */ 76 | StormTopology topology = createTopology(); 77 | if (topology == null) 78 | return -2; 79 | logTopology(LOG, topology); 80 | 81 | /* configure topology */ 82 | Config conf = configureTopology(); 83 | if (conf == null) 84 | return -3; 85 | LOG.info("configuration: " + conf.toString()); 86 | LOG.info("groupid: " + _args.groupid); 87 | LOG.info("topic: " + _args.topic); 88 | 89 | /* run topology */ 90 | if (_args.remoteMode) { 91 | LOG.info("execution mode: remote"); 92 | runInCluster(_args.topologyName, topology, conf); 93 | } 94 | else { 95 | LOG.info("execution mode: local"); 96 | writeSettingsToFile(); 97 | runLocally(_args.topologyName, topology, conf, _args.localRuntime); 98 | } 99 | 100 | return 0; 101 | } 102 | 103 | private StormTopology createTopology() { 104 | TopologyBuilder builder = new TopologyBuilder(); 105 | SchedulingTopologyBuilder builder=new SchedulingTopologyBuilder(); 106 | JoinBolt joinerR = new JoinBolt("R"); 107 | JoinBolt joinerS = new JoinBolt("S"); 108 | 109 | builder.setSpout(KAFKA_SPOUT_ID_R, new KafkaSpout<>(getKafkaSpoutConfig(KAFKA_BROKER, "didiOrder" + _args.dataSize, _args.groupid)), _args.numKafkaSpouts); 110 | builder.setSpout(KAFKA_SPOUT_ID_S, new KafkaSpout<>(getKafkaSpoutConfig(KAFKA_BROKER, "didiGps" + _args.dataSize, _args.groupid)), _args.numKafkaSpouts); 111 | builder.setBolt(SHUFFLE_BOLT_ID, new ShuffleBolt(_args.dataSize), _args.numShufflers) 112 | .shuffleGrouping(KAFKA_SPOUT_ID_R) 113 | .shuffleGrouping(KAFKA_SPOUT_ID_S); 114 | 115 | builder.setDifferentiatedScheduling(SHUFFLE_BOLT_ID, Constraints.relFileds, Constraints.wordFileds); 116 | 117 | builder.setBolt(JOINER_R_BOLT_ID, joinerR, _args.numPartitionsR) 118 | .fieldsGrouping(SCHEDULER_BOLT_ID+builder.getSchedulingNum(), Constraints.nohotRFileds, new Fields(Constraints.wordFileds)) 119 | .fieldsGrouping(SCHEDULER_BOLT_ID+builder.getSchedulingNum(), Constraints.nohotSFileds, new Fields(Constraints.wordFileds)) 120 | .shuffleGrouping(SCHEDULER_BOLT_ID+builder.getSchedulingNum(), Constraints.hotRFileds) 121 | .allGrouping(SCHEDULER_BOLT_ID+builder.getSchedulingNum(), Constraints.hotSFileds); 122 | 123 | builder.setBolt(JOINER_S_BOLT_ID, joinerS, _args.numPartitionsS) 124 | .fieldsGrouping(SCHEDULER_BOLT_ID+builder.getSchedulingNum(), Constraints.nohotSFileds, new Fields(Constraints.wordFileds)) 125 | .fieldsGrouping(SCHEDULER_BOLT_ID+builder.getSchedulingNum(), Constraints.nohotRFileds, new Fields(Constraints.wordFileds)) 126 | .shuffleGrouping(SCHEDULER_BOLT_ID+builder.getSchedulingNum(), Constraints.hotSFileds) 127 | .allGrouping(SCHEDULER_BOLT_ID+builder.getSchedulingNum(), Constraints.hotRFileds); 128 | 129 | 130 | builder.setBolt(POST_PROCESS_BOLT_ID, new PostProcessBolt(_args.numPartitionsR + _args.numPartitionsS), 1) 131 | .globalGrouping(JOINER_R_BOLT_ID, JOINER_TO_POST_STREAM_ID) 132 | .globalGrouping(JOINER_S_BOLT_ID, JOINER_TO_POST_STREAM_ID); 133 | return builder.createTopology(); 134 | } 135 | 136 | private Config configureTopology() { 137 | Config conf = new Config(); 138 | _args.topologyName += "-" + _args.strategy; 139 | conf.setDebug(_args.debug); 140 | conf.setNumWorkers(_args.numWorkers); 141 | conf.setNumAckers(_args.numShufflers); 142 | conf.put("joinFieldIdxR", _args.joinFieldIdxR); 143 | conf.put("joinFieldIdxS", _args.joinFieldIdxS); 144 | conf.put("operator", _args.operator); 145 | 146 | conf.put("fluctuation", _args.fluctuation); 147 | 148 | conf.put("subindexSize", _args.subindexSize); 149 | 150 | conf.put("window", _args.window); 151 | conf.put("winR", _args.winR); 152 | conf.put("winS", _args.winS); 153 | 154 | conf.put("dedup", !_args.noDedup); 155 | conf.put("dedupSize", _args.dedupSize); 156 | 157 | conf.put("aggregate", _args.aggregate); 158 | conf.put("aggReportInSeconds", _args.aggReportInSeconds); 159 | 160 | conf.put("noOutput", _args.noOutput); 161 | conf.put("outputDir", _args.outputDir); 162 | conf.put("simple", _args.simple); 163 | 164 | conf.put("intLower", _args.intLower); 165 | conf.put("intUpper", _args.intUpper); 166 | conf.put("doubleLower", _args.doubleLower); 167 | conf.put("doubleUpper", _args.doubleUpper); 168 | conf.put("charsLength", _args.charsLength); 169 | 170 | return conf; 171 | } 172 | 173 | private void writeSettingsToFile() { 174 | FileWriter output = new FileWriter(_args.outputDir, "top", "txt") 175 | .setPrintStream(System.out); 176 | _args.logArgs(output); 177 | output.endOfFile(); 178 | } 179 | 180 | public static void main(String[] args) throws Exception { 181 | int rc = (new KafkaTopology()).run(args); 182 | LOG.info("return code: " + rc); 183 | } 184 | 185 | 186 | public static KafkaSpoutConfig getKafkaSpoutConfig(String bootstrapServers,String topic, String groupid) { 187 | ByTopicRecordTranslator trans = new ByTopicRecordTranslator<>( 188 | (r) -> new Values(r.topic(), r.partition(), r.offset(), r.key(), r.value()), 189 | new Fields("topic", "partition", "offset", "key", "value")); 190 | return KafkaSpoutConfig.builder(bootstrapServers, new String[]{topic}) 191 | .setProp(ConsumerConfig.GROUP_ID_CONFIG, groupid) 192 | .setRetry(getRetryService()) 193 | .setRecordTranslator(trans) 194 | .setFirstPollOffsetStrategy(EARLIEST) 195 | .setProcessingGuarantee(KafkaSpoutConfig.ProcessingGuarantee.AT_MOST_ONCE) 196 | .build(); 197 | } 198 | public static KafkaSpoutRetryService getRetryService() { 199 | return new KafkaSpoutRetryExponentialBackoff(KafkaSpoutRetryExponentialBackoff.TimeInterval.microSeconds(0), 200 | KafkaSpoutRetryExponentialBackoff.TimeInterval.milliSeconds(2), Integer.MAX_VALUE, KafkaSpoutRetryExponentialBackoff.TimeInterval.seconds(10)); 201 | } 202 | 203 | } -------------------------------------------------------------------------------- /src/main/java/com/basic/core/SchedulingTopologyBuilder.java: -------------------------------------------------------------------------------- 1 | package com.basic.core; 2 | 3 | import com.basic.core.bolt.CoinBolt; 4 | import com.basic.core.bolt.PredictorBolt; 5 | import com.basic.core.bolt.SchedulerBolt; 6 | import org.apache.storm.topology.TopologyBuilder; 7 | 8 | /** 9 | * locate com.basic.core 10 | * Created by windy on 2019/1/2. 11 | */ 12 | public class SchedulingTopologyBuilder extends TopologyBuilder { 13 | 14 | private int schedulingNum=0; 15 | 16 | public void setDifferentiatedScheduling(String UPStreamCompoentID,String UPStreamCompoentRelFields, String UPStreamCompoentKeyFields){ 17 | SchedulerBolt schedulerBolt=new SchedulerBolt(UPStreamCompoentID,UPStreamCompoentRelFields,UPStreamCompoentKeyFields); 18 | CoinBolt coinBolt=new CoinBolt(); 19 | PredictorBolt predictorBolt=new PredictorBolt(); 20 | this.setBolt(Constraints.SCHEDULER_BOLT_ID+schedulingNum, schedulerBolt, 32).shuffleGrouping(UPStreamCompoentID).allGrouping(Constraints.PREDICTOR_BOLT_ID+schedulingNum); 21 | this.setBolt(Constraints.COIN_BOLT_ID+schedulingNum, coinBolt, 32).shuffleGrouping(Constraints.SCHEDULER_BOLT_ID+schedulingNum, Constraints.coinFileds); 22 | this.setBolt(Constraints.PREDICTOR_BOLT_ID+schedulingNum, predictorBolt,1).globalGrouping(Constraints.COIN_BOLT_ID+schedulingNum); 23 | schedulingNum++; 24 | } 25 | 26 | public int getSchedulingNum() { 27 | return schedulingNum-1; 28 | } 29 | 30 | } 31 | -------------------------------------------------------------------------------- /src/main/java/com/basic/core/TopologyArgs.java: -------------------------------------------------------------------------------- 1 | package com.basic.core; 2 | 3 | import java.io.PrintStream; 4 | import java.io.Serializable; 5 | import java.util.List; 6 | 7 | //import static org.apache.commons.io.FilenameUtils.normalizeNoEndSeparator; 8 | import org.kohsuke.args4j.Argument; 9 | import org.kohsuke.args4j.CmdLineException; 10 | import org.kohsuke.args4j.CmdLineParser; 11 | import org.kohsuke.args4j.Option; 12 | import org.slf4j.Logger; 13 | 14 | //import static org.apache.storm.shade.org.apache.commons.io.FilenameUtils.normalizeNoEndSeparator; 15 | import static org.apache.commons.io.FilenameUtils.normalizeNoEndSeparator; 16 | import static org.slf4j.LoggerFactory.getLogger; 17 | 18 | import static com.google.common.base.Preconditions.checkState; 19 | import com.google.common.collect.ImmutableList; 20 | import static com.google.common.collect.Lists.newLinkedList; 21 | 22 | import com.basic.core.util.FileWriter; 23 | import static com.basic.core.util.SchemaUtils.getFieldIdx; 24 | import static com.basic.core.util.TimeUtils.getTimestamp; 25 | 26 | public class TopologyArgs implements Serializable 27 | { 28 | private static final Logger LOG = getLogger(TopologyArgs.class); 29 | 30 | private static final int SCREEN_WIDTH = 80; 31 | 32 | private final String _mainclassName; 33 | 34 | public TopologyArgs(String className) { 35 | _mainclassName = className; 36 | } 37 | 38 | private static final String R_SCHEMA = "r1, r2, r3"; 39 | private static final String S_SCHEMA = "s1, s2, s3, s4"; 40 | private static final String R_JOIN_FIELD = "r1"; 41 | private static final String S_JOIN_FIELD = "s1"; 42 | private static final String JOIN_OPERATOR = "="; 43 | 44 | private static final String TOPOLOGY_NAME = "Simois"; 45 | 46 | private static final int DEFAULT_INT_LOWER = 0; 47 | private static final int DEFAULT_INT_UPPER = 20000; 48 | private static final int DEFAULT_DOUBLE_LOWER = 0; 49 | private static final int DEFAULT_DOUBLE_UPPER = 20000; 50 | private static final int DEFAULT_CHARS_LENGTH = 20; 51 | 52 | public static final String HASH_STRATEGY = "hash"; 53 | public static final String RANDOM_STRATEGY = "random"; 54 | public static final String FAST_JOIN_STRATEGY = "fastjoin"; 55 | public static final String CONTRAND_STRATEGY = "contrand"; 56 | @Option(name = "-R", aliases = "--schema-r", metaVar = "", 57 | usage = "schema of relation R [def:\"" + R_SCHEMA + "\"]") 58 | public String schemaR = R_SCHEMA; 59 | 60 | @Option(name = "-S", aliases = "--schema-s", metaVar = "", 61 | usage = "schema of relation R [def:\"" + S_SCHEMA + "\"]") 62 | public String schemaS = S_SCHEMA; 63 | 64 | @Option(name = "-jr", aliases = "--join-r", metaVar = "", 65 | depends = "-R", usage = "join field of relation R [def:\"" 66 | + R_JOIN_FIELD + "\"]" + "\nDEPENDENCY: -R") 67 | public String joinFieldR = R_JOIN_FIELD; 68 | 69 | @Option(name = "-js", aliases = "--join-s", metaVar = "", 70 | depends = "-S", usage = "join field of relation S [def:\"" 71 | + S_JOIN_FIELD + "\"]" + "\nDEPENDENCY: -S") 72 | public String joinFieldS = S_JOIN_FIELD; 73 | 74 | @Option(name = "-op", aliases = "--operator", metaVar = "", 75 | usage = "join operator [def:\"" + JOIN_OPERATOR + "\"]") 76 | public String operator = JOIN_OPERATOR; 77 | 78 | @Option(name = "--name", metaVar = "", usage = "topology name [def:" 79 | + TOPOLOGY_NAME + "]") 80 | public String topologyName = TOPOLOGY_NAME; 81 | 82 | @Option(name = "--remote", 83 | usage = "run topology in cluster (remote mode) [def:true]") 84 | public boolean remoteMode = true; 85 | 86 | @Option(name = "-lrt", aliases = "--local-runtime", metaVar = "", 87 | usage = "running time (in seconds) for local mode [def:5]") 88 | public int localRuntime = 5; 89 | 90 | @Option(name = "--debug", usage = "enable debug logs [def:false]") 91 | public boolean debug = false; 92 | 93 | @Option(name = "-n", aliases = "--worker", metaVar = "", 94 | usage = "# workers [def:2]") 95 | public int numWorkers = 2; 96 | 97 | // @Option(name = "-rs", aliases = "--source-rs", metaVar = "", 98 | // usage = "# random data sources (mixing relations) [def:1]") 99 | // public int numGenerators = 1; 100 | // 101 | // @Option(name = "-r", aliases = "--source-r", metaVar = "", 102 | // usage = "# random data sources of relation R [def:0]") 103 | // public int numGeneratorsR = 0; 104 | // 105 | // @Option(name = "-s", aliases = "--source-s", metaVar = "", 106 | // usage = "# random data sources of relation S [def:0]") 107 | // public int numGeneratorsS = 0; 108 | @Option(name = "-ks", aliases = "--source-rs", metaVar = "", 109 | usage = "# random data sources (mixing relations) [def:1]") 110 | public int numKafkaSpouts = 2; 111 | 112 | @Option(name = "-f1", aliases = "--shuffler", metaVar = "", 113 | usage = "# shufflers [def:2]") 114 | public int numShufflers = 4; 115 | 116 | @Option(name = "-f2", aliases = "--reshuffler", metaVar = "", 117 | usage = "# reshufflers [def:4]") 118 | public int numReshufflers = 4; 119 | 120 | @Option(name = "-pr", aliases = "--part-r", metaVar = "", 121 | usage = "# partitions of relation R [def:4]") 122 | public int numPartitionsR = 8; 123 | 124 | @Option(name = "-ps", aliases = "--part-s", metaVar = "", 125 | usage = "# partitions of relation S [def:4]") 126 | public int numPartitionsS = 8; 127 | 128 | @Option(name = "-g", aliases = "--gatherer", metaVar = "", 129 | usage = "# result gatherers [def:1]") 130 | public int numGatherers = 1; 131 | 132 | @Option(name = "-trs", aliases = "--tps-rs", metaVar = "", 133 | usage = "tuples per second generated by each random data source" 134 | + " (mixing relations) [def:20]") 135 | public Long tuplesPerSecond = 20L; 136 | 137 | @Option(name = "-tr", aliases = "--tps-r", metaVar = "", 138 | usage = "tuples per second generated by each random data source" 139 | + " of relation R [def:10]") 140 | public Long tuplesPerSecondR = 10L; 141 | 142 | @Option(name = "-ts", aliases = "--tps-s", metaVar = "", 143 | usage = "tuples per second generated by each random data source" 144 | + " of relation S [def:10]") 145 | public Long tuplesPerSecondS = 10L; 146 | 147 | @Option(name = "-fluc", aliases = "--fluctuate", 148 | usage = "enable fluctuation for random data sources [def:false]") 149 | public Boolean fluctuation = false; 150 | 151 | @Option(name = "--index-size", metaVar = "", hidden = false, 152 | usage = "size (in records) of each sub-index [def:2048]") 153 | public Integer subindexSize = 2048; 154 | 155 | @Option(name = "-win", aliases = "--window", 156 | usage = "enable sliding window [def:false]") 157 | public Boolean window = false; 158 | 159 | @Option(name = "-wr", aliases = "--window-r", metaVar = "", 160 | usage = "window size (in seconds) for stream R [def:2]") 161 | public double winInSecR = 2; 162 | 163 | @Option(name = "-ws", aliases = "--window-s", metaVar = "", 164 | usage = "window size (in seconds) for stream S [def:2]") 165 | public double winInSecS = 2; 166 | 167 | @Option(name = "--no-dedup", hidden = true, 168 | usage = "disable deduplication in the post-processing " 169 | + "stage [def:false]") 170 | public Boolean noDedup = false; 171 | 172 | @Option(name = "--dedup-size", metaVar = "", hidden = true, 173 | usage = "size (in number of records) of hash set for " 174 | + "deduplication in the post-processing stage [def:500000]") 175 | public int dedupSize = 500000; 176 | 177 | @Option(name = "-agg", aliases = "--aggregate", 178 | usage = "enable aggregation [def:false]") 179 | public Boolean aggregate = false; 180 | 181 | @Option(name = "-ai", aliases = "--aggregate-interval", metaVar = "", 182 | usage = "aggregation report interval (in seconds) [def:1]") 183 | public Long aggReportInSeconds = 1L; 184 | 185 | @Option(name = "--no-output", 186 | usage = "disable output of join results [def:false]") 187 | public Boolean noOutput = false; 188 | 189 | @Option(name = "-d", aliases = "--output-dir", metaVar = "", 190 | usage = "output directory [def:null]") 191 | public String outputDir = null; 192 | 193 | 194 | @Option(name = "-x", aliases = "--topic") 195 | public String topic = "didi"; 196 | @Option(name = "-z", aliases = "--groupid") 197 | public String groupid = "fastjoin"; 198 | 199 | 200 | @Option(name = "--simple-output", 201 | usage = "enable simple output format for join results [def:false]") 202 | public Boolean simple = false; 203 | 204 | @Option(name = "-il", aliases = "--int-lower", metaVar = "", 205 | usage = "lower bound of random integer [def:" + DEFAULT_INT_LOWER 206 | + "]") 207 | public Integer intLower = DEFAULT_INT_LOWER; 208 | 209 | @Option(name = "-iu", aliases = "--int-upper", metaVar = "", 210 | usage = "upper bound of random integer [def:" + DEFAULT_INT_UPPER 211 | + "]") 212 | public Integer intUpper = DEFAULT_INT_UPPER; 213 | 214 | @Option(name = "-dl", aliases = "--double-lower", metaVar = "", 215 | usage = "lower bound of random double [def:" + DEFAULT_DOUBLE_LOWER 216 | + "]") 217 | public Integer doubleLower = DEFAULT_DOUBLE_LOWER; 218 | 219 | @Option(name = "-du", aliases = "--double-upper", metaVar = "", 220 | usage = "upper bound of random double [def:" + DEFAULT_DOUBLE_UPPER 221 | + "]") 222 | public Integer doubleUpper = DEFAULT_DOUBLE_UPPER; 223 | 224 | @Option(name = "-c", aliases = "--chars-length", metaVar = "", 225 | usage = "length of random chars [def:" + DEFAULT_CHARS_LENGTH + "]") 226 | public Integer charsLength = DEFAULT_CHARS_LENGTH; 227 | 228 | @Option(name = "-h", aliases = { "-?", "--help" }, hidden = false, 229 | help = true, usage = "print this help message") 230 | public boolean help = false; 231 | 232 | @Option(name = "--s", metaVar = "", usage = "strategy [def:" 233 | + HASH_STRATEGY + "]") 234 | public String strategy = HASH_STRATEGY; 235 | 236 | @Option(name = "--size", metaVar = "", usage = "size [def:" 237 | + "10G" + "]") 238 | public String dataSize = "10G"; 239 | 240 | @Option(name = "-st", aliases = "--startTime", metaVar = "", 241 | usage = "start time of migration [def:" + "5000" 242 | + "]") 243 | public Integer startTime = 5000; 244 | 245 | 246 | @Option(name = "-t", aliases = "--threshold", metaVar = "", 247 | usage = "threshold of load imbalance[def:" + "5000" 248 | + "]") 249 | public double threshold = 2.5; 250 | 251 | @Option(name = "-i", aliases = "--interval", metaVar = "", 252 | usage = "interval of load imbalance detection[def:" + "1000" 253 | + "]") 254 | public double interval = 1000; 255 | 256 | @Option(name = "-rm", aliases = "--rm", metaVar = "", 257 | usage = "members in each group which stores r tuple[def:" + "2" 258 | + "]") 259 | public int rMembers = 2; 260 | 261 | @Option(name = "-sm", aliases = "--sm", metaVar = "", 262 | usage = "members in each group which stores s tuple[def:" + "2" 263 | + "]") 264 | public int sMembers = 2; 265 | 266 | public int joinFieldIdxR; 267 | public int joinFieldIdxS; 268 | public Long winR; 269 | public Long winS; 270 | 271 | public boolean processArgs(String[] args) throws Exception { 272 | if (!parseArgs(args)) 273 | return false; 274 | 275 | if (help) { 276 | printHelp(System.out); 277 | } 278 | else { 279 | sanityCheckArgs(); 280 | deriveArgs(); 281 | } 282 | 283 | return true; 284 | } 285 | 286 | private boolean parseArgs(String[] args) { 287 | CmdLineParser parser = new CmdLineParser(this); 288 | parser.getProperties().withUsageWidth(SCREEN_WIDTH); 289 | 290 | try { 291 | parser.parseArgument(args); 292 | } 293 | catch (CmdLineException e) { 294 | System.err.println("ERROR: " + e.getMessage()); 295 | System.err.println(); 296 | printHelp(System.err); 297 | 298 | return false; 299 | } 300 | 301 | return true; 302 | } 303 | 304 | private void sanityCheckArgs() { 305 | joinFieldR = joinFieldR.trim(); 306 | checkState(schemaR.contains(joinFieldR), 307 | "Schema R does not contain field " + joinFieldR); 308 | 309 | joinFieldS = joinFieldS.trim(); 310 | checkState(schemaS.contains(joinFieldS), 311 | "Schema S does not contain field " + joinFieldS); 312 | 313 | operator = operator.trim(); 314 | checkState( 315 | ImmutableList.of("=").contains( 316 | operator), "Invalid join operator: " + operator); 317 | 318 | checkState(numWorkers > 0, "At least one worker is required"); 319 | // checkState(numGenerators >= 0, 320 | // "Non-negative number of sources R&S is required"); 321 | // checkState(numGeneratorsR >= 0, 322 | // "Non-negative number of sources R is required"); 323 | // checkState(numGeneratorsS >= 0, 324 | // "Non-negative number of sources S is required"); 325 | checkState(numShufflers > 0, "At least one shuffler is required"); 326 | checkState(numReshufflers > 0, "At least one reshuffler is required"); 327 | checkState(numPartitionsR > 0, "At least one partition R is required"); 328 | checkState(numPartitionsS > 0, "At least one partition S is required"); 329 | checkState(numGatherers > 0, "At least one result gatherer is required"); 330 | 331 | checkState(tuplesPerSecond >= 0, "TPS of source R&S cannot be negative"); 332 | checkState(tuplesPerSecondR >= 0, "TPS of source R cannot be negative"); 333 | checkState(tuplesPerSecondS >= 0, "TPS of source S cannot be negative"); 334 | 335 | checkState(subindexSize > 0, "Size of each sub-index must be positive"); 336 | 337 | checkState(!window || winInSecR > 0, 338 | "Window size for stream R must be positive"); 339 | checkState(!window || winInSecS > 0, 340 | "Window size for stream S must be positive"); 341 | 342 | checkState(noDedup || dedupSize > 0, "Dedup set size must be positive"); 343 | 344 | checkState(aggReportInSeconds > 0, 345 | "Aggregation report interval must be at least 1 second"); 346 | 347 | checkState(intUpper > intLower); 348 | checkState(doubleUpper > doubleLower); 349 | checkState(charsLength > 0, "length of chars must be positive"); 350 | } 351 | 352 | private void deriveArgs() { 353 | if (outputDir != null) { 354 | outputDir = normalizeNoEndSeparator(outputDir); 355 | if (outputDir.isEmpty()) { 356 | outputDir = "."; 357 | } 358 | else { 359 | outputDir += "_" + getTimestamp(); 360 | } 361 | } 362 | 363 | joinFieldIdxR = getFieldIdx(schemaR, joinFieldR); 364 | joinFieldIdxS = getFieldIdx(schemaS, joinFieldS); 365 | 366 | winR = (long) (winInSecR * 1000); 367 | winS = (long) (winInSecS * 1000); 368 | } 369 | 370 | public void showArgs() { 371 | for (String msg : getInfo()) { 372 | System.out.println(msg); 373 | } 374 | } 375 | 376 | public void logArgs() { 377 | for (String msg : getInfo()) { 378 | LOG.info(msg); 379 | } 380 | } 381 | 382 | public void logArgs(FileWriter output) { 383 | for (String msg : getInfo()) { 384 | output.write(msg); 385 | } 386 | } 387 | 388 | private List getInfo() { 389 | List info = newLinkedList(); 390 | 391 | info.add(" schema R: [" + schemaR + "]; join_field_index: " 392 | + joinFieldIdxR); 393 | info.add(" schema S: [" + schemaS + "]; join_field_index: " 394 | + joinFieldIdxS); 395 | info.add(" join predicate: R." + joinFieldR + " " + operator + " S." 396 | + joinFieldS); 397 | 398 | info.add(" topology name: " + topologyName); 399 | info.add(" remote mode: " + remoteMode); 400 | info.add(" local runtime: " + localRuntime + " sec"); 401 | info.add(" debug: " + debug); 402 | 403 | info.add(" # workers: " + numWorkers); 404 | 405 | info.add(" # shufflers: " + numShufflers); 406 | info.add(" # partitions R: " + numPartitionsR); 407 | info.add(" # partitions S: " + numPartitionsS); 408 | info.add(" # result gatherers: " + numGatherers); 409 | 410 | info.add(" tps of each source R&S: " + tuplesPerSecond); 411 | info.add(" tps of each source R: " + tuplesPerSecondR); 412 | info.add(" tps of each source S: " + tuplesPerSecondS); 413 | info.add(" fluctuation: " + fluctuation); 414 | 415 | info.add(" size of each sub-index: " + subindexSize); 416 | 417 | info.add(" sliding window: " + window); 418 | info.add(" window size for R: " + winR + " ms"); 419 | info.add(" window size for S: " + winS + " ms"); 420 | info.add(" sub-index size: " + subindexSize + " tuples"); 421 | 422 | info.add(" deduplication: " + !noDedup); 423 | info.add(" dedup set size: " + dedupSize + " items"); 424 | 425 | info.add(" aggregation: " + aggregate); 426 | info.add(" aggregation report interval: " + aggReportInSeconds 427 | + " sec"); 428 | 429 | info.add(" output: " + !noOutput); 430 | if (outputDir != null) { 431 | info.add(" output dir: " + outputDir); 432 | } 433 | info.add(" simple output: " + simple); 434 | 435 | return info; 436 | } 437 | 438 | private void printHelp(PrintStream out) { 439 | String cmd = "java " + _mainclassName; 440 | String cmdIndent = " " + cmd; 441 | 442 | out.println("USAGE: "); 443 | out.println(cmdIndent + " [OPTION]..."); 444 | out.println(); 445 | out.println("OPTIONS: "); 446 | (new CmdLineParser(this)).printUsage(out); 447 | out.println(); 448 | } 449 | 450 | public static int main(String[] args) throws Exception { 451 | TopologyArgs allArgs = new TopologyArgs("TopologyArgs"); 452 | 453 | if (!allArgs.processArgs(args)) 454 | return -1; 455 | 456 | if (!allArgs.help) 457 | allArgs.showArgs(); 458 | 459 | return 0; 460 | } 461 | } 462 | -------------------------------------------------------------------------------- /src/main/java/com/basic/core/bolt/AggregatorBolt.java: -------------------------------------------------------------------------------- 1 | package com.basic.core.bolt; 2 | 3 | import java.util.Map; 4 | import static java.util.concurrent.TimeUnit.SECONDS; 5 | 6 | import org.apache.storm.task.OutputCollector; 7 | import org.apache.storm.task.TopologyContext; 8 | import org.apache.storm.topology.OutputFieldsDeclarer; 9 | import org.apache.storm.topology.base.BaseRichBolt; 10 | import org.apache.storm.tuple.Tuple; 11 | import org.apache.storm.tuple.Values; 12 | import org.slf4j.Logger; 13 | import static org.slf4j.LoggerFactory.getLogger; 14 | 15 | import static com.google.common.collect.Maps.newHashMap; 16 | 17 | //import backtype.storm.task.OutputCollector; 18 | //import backtype.storm.task.TopologyContext; 19 | //import backtype.storm.topology.OutputFieldsDeclarer; 20 | //import backtype.storm.topology.base.BaseRichBolt; 21 | //import backtype.storm.tuple.Tuple; 22 | //import backtype.storm.tuple.Values; 23 | import com.basic.core.util.FileWriter; 24 | import com.basic.core.util.Stopwatch; 25 | import static com.basic.core.util.CastUtils.getBoolean; 26 | import static com.basic.core.util.CastUtils.getLong; 27 | import static com.basic.core.util.CastUtils.getString; 28 | 29 | public class AggregateBolt extends BaseRichBolt 30 | { 31 | private static final Logger LOG = getLogger(AggregateBolt.class); 32 | 33 | private FileWriter _output; 34 | 35 | private boolean _aggregate; 36 | 37 | private long _count; 38 | private long _sum; 39 | private double _avg; 40 | private long _min; 41 | private long _max; 42 | private Map _ledger; 43 | 44 | private long _aggReportInSeconds; 45 | private long _triggerEmitInSeconds; 46 | private Stopwatch _stopwatch; 47 | 48 | @Override 49 | public void prepare(Map conf, TopologyContext context, 50 | OutputCollector collector) { 51 | String outputDir = getString(conf.get("outputDir")); 52 | String prefix = "a" + context.getThisTaskId(); 53 | _output = (new FileWriter(outputDir, prefix, "out")).setFlushSize(10) 54 | .setPrintStream(System.out); 55 | 56 | _aggregate = getBoolean(conf.get("aggregate")); 57 | _count = 0; 58 | if (_aggregate) { 59 | _sum = 0; 60 | _avg = 0.0; 61 | _min = Long.MAX_VALUE; 62 | _max = Long.MIN_VALUE; 63 | } 64 | _ledger = newHashMap(); 65 | 66 | _aggReportInSeconds = getLong(conf.get("aggReportInSeconds")); 67 | _triggerEmitInSeconds = _aggReportInSeconds; 68 | 69 | LOG.info("aggregate:" + _aggregate + ", report_interval(sec):" 70 | + _aggReportInSeconds); 71 | 72 | _stopwatch = Stopwatch.createStarted(); 73 | } 74 | 75 | public void execute(Tuple tuple) { 76 | aggregate(tuple); 77 | 78 | if (isTimeToOutputAggregate()) { 79 | output(getAggregate()); 80 | } 81 | } 82 | 83 | public void declareOutputFields(OutputFieldsDeclarer declarer) { 84 | // this bolt does not emit anything 85 | } 86 | 87 | public void cleanup() { 88 | _stopwatch.stop(); 89 | 90 | output(getAggregate()); 91 | 92 | if (_output != null) 93 | _output.endOfFile(); 94 | } 95 | 96 | private void aggregate(Tuple tuple) { 97 | /* retrieve the ledger record */ 98 | Integer srcId = Integer.valueOf(tuple.getSourceTask()); 99 | Values values = _ledger.get(srcId); 100 | if (values == null) { 101 | values = new Values(Long.valueOf(0), Long.valueOf(0)); 102 | } 103 | 104 | /* update count */ 105 | long countOld = getLong(values, 0); 106 | long countNew = tuple.getLongByField("count"); 107 | _count += countNew - countOld; 108 | values.set(0, Long.valueOf(countNew)); 109 | 110 | /* update sum, avg, min and max */ 111 | if (_aggregate) { 112 | long sumOld = getLong(values, 1); 113 | long sumNew = tuple.getLongByField("sum"); 114 | _sum += sumNew - sumOld; 115 | values.set(1, Long.valueOf(sumNew)); 116 | 117 | _avg = ((double) _sum) / _count; 118 | 119 | _min = Math.min(_min, tuple.getLongByField("min")); 120 | _max = Math.max(_max, tuple.getLongByField("max")); 121 | } 122 | 123 | /* update the ledger record */ 124 | _ledger.put(srcId, values); 125 | } 126 | 127 | private boolean isTimeToOutputAggregate() { 128 | long currTime = _stopwatch.elapsed(SECONDS); 129 | 130 | if (currTime >= _triggerEmitInSeconds) { 131 | _triggerEmitInSeconds = currTime + _aggReportInSeconds; 132 | return true; 133 | } 134 | else { 135 | return false; 136 | } 137 | } 138 | 139 | private String getAggregate() { 140 | StringBuilder sb = new StringBuilder(); 141 | 142 | sb.append("[GLOBAL AGG @ " + _stopwatch.elapsed(SECONDS) + " sec]"); 143 | sb.append(" count=" + _count); 144 | if (_aggregate) { 145 | sb.append(", sum=" + _sum); 146 | sb.append(", avg=" + _avg); 147 | sb.append(", min=" + _min); 148 | sb.append(", max=" + _max); 149 | } 150 | 151 | return sb.toString(); 152 | } 153 | 154 | private void output(String msg) { 155 | if (_output != null) 156 | _output.write(msg); 157 | } 158 | } -------------------------------------------------------------------------------- /src/main/java/com/basic/core/bolt/CoinBolt.java: -------------------------------------------------------------------------------- 1 | package com.basic.core.bolt; 2 | 3 | import com.basic.core.Constraints; 4 | import com.basic.core.util.PredictHeavyLoadKeyUtil; 5 | import org.apache.storm.task.OutputCollector; 6 | import org.apache.storm.task.TopologyContext; 7 | import org.apache.storm.topology.OutputFieldsDeclarer; 8 | import org.apache.storm.topology.base.BaseRichBolt; 9 | import org.apache.storm.tuple.Fields; 10 | import org.apache.storm.tuple.Tuple; 11 | import org.apache.storm.tuple.Values; 12 | import java.util.Map; 13 | import static com.basic.core.Constraints.Threshold_r; 14 | 15 | /** 16 | * locate com.basic.storm.bolt 17 | * Created by windy on 2019/1/2 18 | */ 19 | public class CoinBolt extends BaseRichBolt { 20 | private OutputCollector collector; 21 | private PredictHeavyLoadKeyUtil predictHeavyLoadKeyUtil=PredictHeavyLoadKeyUtil.getPredictHeavyLoadKeyUtilInstance(); 22 | 23 | public void prepare(Map conf, TopologyContext context, OutputCollector collector) { 24 | this.collector = collector; 25 | } 26 | 27 | public void execute(Tuple tuple) { 28 | String rel = tuple.getStringByField(Constraints.relFileds); 29 | String word = tuple.getStringByField(Constraints.wordFileds); 30 | Integer basecount = tuple.getIntegerByField(Constraints.coinCountFileds); 31 | int coincount = predictHeavyLoadKeyUtil.countCointUtilUp(); 32 | if(coincount+basecount>=Threshold_r) 33 | collector.emit(new Values(rel,word,coincount)); 34 | collector.ack(tuple); 35 | } 36 | 37 | public void declareOutputFields(OutputFieldsDeclarer declarer) { 38 | declarer.declare(new Fields(Constraints.relFileds,Constraints.wordFileds,Constraints.coinCountFileds)); 39 | } 40 | 41 | } 42 | -------------------------------------------------------------------------------- /src/main/java/com/basic/core/bolt/JoinBolt.java: -------------------------------------------------------------------------------- 1 | package com.basic.core.bolt; 2 | 3 | import java.text.DecimalFormat; 4 | import java.util.Collection; 5 | import java.util.List; 6 | import java.util.Map; 7 | import java.util.Queue; 8 | import static java.util.concurrent.TimeUnit.SECONDS; 9 | 10 | import org.apache.commons.lang3.tuple.ImmutablePair; 11 | import org.apache.commons.lang3.tuple.Pair; 12 | import org.apache.storm.task.TopologyContext; 13 | import org.apache.storm.topology.BasicOutputCollector; 14 | import org.apache.storm.topology.OutputFieldsDeclarer; 15 | import org.apache.storm.topology.base.BaseBasicBolt; 16 | import org.apache.storm.tuple.Fields; 17 | import org.apache.storm.tuple.Tuple; 18 | import org.apache.storm.tuple.Values; 19 | import org.slf4j.Logger; 20 | import static org.slf4j.LoggerFactory.getLogger; 21 | 22 | import static com.google.common.base.Preconditions.checkState; 23 | import com.google.common.collect.ImmutableList; 24 | import com.google.common.collect.LinkedListMultimap; 25 | import com.google.common.collect.Multimap; 26 | import static com.google.common.collect.Lists.newLinkedList; 27 | 28 | //import backtype.storm.task.TopologyContext; 29 | //import backtype.storm.topology.BasicOutputCollector; 30 | //import backtype.storm.topology.OutputFieldsDeclarer; 31 | //import backtype.storm.topology.base.BaseBasicBolt; 32 | //import backtype.storm.tuple.Fields; 33 | //import backtype.storm.tuple.Tuple; 34 | //import backtype.storm.tuple.Values; 35 | import com.basic.core.util.FileWriter; 36 | import com.basic.core.util.Stopwatch; 37 | import static com.basic.core.util.CastUtils.getBoolean; 38 | import static com.basic.core.util.CastUtils.getInt; 39 | import static com.basic.core.util.CastUtils.getList; 40 | import static com.basic.core.util.CastUtils.getLong; 41 | import static com.basic.core.util.CastUtils.getString; 42 | 43 | public class JoinBolt extends BaseBasicBolt 44 | { 45 | 46 | private static final List SCHEMA = ImmutableList.of("relation", 47 | "timestamp", "key", "value"); 48 | private static final long PROFILE_REPORT_PERIOD_IN_SEC = 1; 49 | private static final int BYTES_PER_TUPLE_R = 64; 50 | private static final int BYTES_PER_TUPLE_S = 56; 51 | private static final int BYTES_PER_MB = 1024 * 1024; 52 | 53 | private static final Logger LOG = getLogger(JoinBolt.class); 54 | 55 | private final String _rel; 56 | 57 | public JoinBolt(String relation) { 58 | super(); 59 | _rel = relation; 60 | 61 | checkState(_rel.equals("R") || _rel.equals("S"), "Unknown relation: " 62 | + _rel); 63 | } 64 | 65 | private int _tid; 66 | 67 | private FileWriter _output; 68 | 69 | private int _thisJoinFieldIdx; 70 | private int _oppJoinFieldIdx; 71 | private String _operator; 72 | 73 | private boolean _window; 74 | private long _thisWinSize; 75 | private long _oppWinSize; 76 | 77 | private int _subindexSize; 78 | private Queue _indexQueue; 79 | private Multimap _currMap; 80 | 81 | private long _profileReportInSeconds; 82 | private long _triggerReportInSeconds; 83 | private Stopwatch _stopwatch; 84 | private DecimalFormat _df; 85 | 86 | private long _tuplesStored; 87 | private long _tuplesJoined; 88 | private int _thisTupleSize; 89 | private int _oppTupleSize; 90 | 91 | @Override 92 | public void prepare(Map conf, TopologyContext context) { 93 | _tid = context.getThisTaskId(); 94 | 95 | String outputDir = getString(conf.get("outputDir")); 96 | String prefix = "srj_joiner_" + _rel.toLowerCase() + _tid; 97 | // _output = (new FileWriter(outputDir, prefix, "txt")).setFlushSize(10) 98 | // .setPrintStream(System.out); 99 | _output = new FileWriter("/tmp/", prefix, "txt"); 100 | _subindexSize = getInt(conf.get("subindexSize")); 101 | 102 | LOG.info("relation:" + _rel + ", join_field_idx(this):" 103 | + _thisJoinFieldIdx + ", join_field_idx(opp):" 104 | + _oppJoinFieldIdx + ", operator:" + _operator + ", window:" 105 | + _window + ", win_size:" + _thisWinSize + ", subindex_size:" 106 | + _subindexSize); 107 | 108 | /* indexes */ 109 | _indexQueue = newLinkedList(); 110 | _currMap = LinkedListMultimap.create(_subindexSize); 111 | 112 | /* profiling */ 113 | _tuplesStored = 0; 114 | _tuplesJoined = 0; 115 | 116 | _df = new DecimalFormat("0.00"); 117 | _profileReportInSeconds = PROFILE_REPORT_PERIOD_IN_SEC; 118 | _triggerReportInSeconds = _profileReportInSeconds; 119 | _stopwatch = Stopwatch.createStarted(); 120 | } 121 | 122 | public void execute(Tuple tuple, BasicOutputCollector collector) { 123 | String rel = tuple.getStringByField("relation"); 124 | if (!rel.equals("R") && !rel.equals("S")) { 125 | LOG.error("Unknown relation: " + rel); 126 | return; 127 | } 128 | 129 | if (rel.equals(_rel)) { 130 | store(tuple); 131 | ++_tuplesStored; 132 | collector.emit(new Values(tuple.getStringByField("relation") 133 | , tuple.getLongByField("timestamp") 134 | , tuple.getIntegerByField("key") 135 | , tuple.getStringByField("value"))); 136 | } 137 | else { // rel.equals(Opp(_rel)) 138 | join(tuple, collector); 139 | ++_tuplesJoined; 140 | } 141 | 142 | // if (isTimeToOutputProfile()) { 143 | // output(getProfile()); 144 | // } 145 | } 146 | 147 | public void declareOutputFields(OutputFieldsDeclarer declarer) { 148 | declarer.declare(new Fields(SCHEMA)); 149 | } 150 | 151 | @Override 152 | public void cleanup() { 153 | _stopwatch.stop(); 154 | 155 | StringBuilder sb = new StringBuilder(); 156 | 157 | sb.append("relation:" + _rel); 158 | sb.append(", num_of_indexes:" + (_indexQueue.size() + 1)); 159 | 160 | output(sb.toString()); 161 | 162 | if (_output != null) 163 | _output.endOfFile(); 164 | } 165 | 166 | private void store(Tuple tuple) { 167 | Long ts = tuple.getLongByField("timestamp"); 168 | int key = tuple.getIntegerByField("key"); 169 | String value = tuple.getStringByField("value"); 170 | 171 | Values values = new Values(ts, key, value); 172 | 173 | _currMap.put(key, values); 174 | 175 | if (_currMap.size() >= _subindexSize) { 176 | _indexQueue.offer(ImmutablePair.of(ts, _currMap)); 177 | _currMap = LinkedListMultimap.create(_subindexSize); 178 | } 179 | } 180 | 181 | private void join(Tuple tupleOpp, BasicOutputCollector collector) { 182 | /* join with archive indexes */ 183 | int numToDelete = 0; 184 | long tsOpp = tupleOpp.getLongByField("timestamp"); 185 | for (Pair pairTsIndex : _indexQueue) { 186 | long ts = getLong(pairTsIndex.getLeft()); 187 | if (_window && !isInWindow(tsOpp, ts)) { 188 | ++numToDelete; 189 | continue; 190 | } 191 | 192 | join(tupleOpp, pairTsIndex.getRight(), collector); 193 | } 194 | 195 | for (int i = 0; i < numToDelete; ++i) { 196 | _indexQueue.poll(); 197 | } 198 | 199 | /* join with current index */ 200 | join(tupleOpp, _currMap, collector); 201 | } 202 | 203 | private void join(Tuple tupleOpp, Object index, 204 | BasicOutputCollector collector) { 205 | int key = tupleOpp.getIntegerByField("key"); 206 | 207 | for (Values record : getMatchings(index, key)) { 208 | if (_rel.equals("R")) { 209 | output("R: " + value + " ---- " + tupleOpp.getStringByField("value")); 210 | } 211 | else { // _rel.equals("S") 212 | output("S: " + tupleOpp.getStringByField("value") + " ---- " + value); 213 | } 214 | } 215 | } 216 | 217 | @SuppressWarnings("unchecked") 218 | private Collection getMatchings(Object index, Object value) { 219 | return ((Multimap) index).get(value); 220 | } 221 | 222 | @SuppressWarnings("unchecked") 223 | private int getIndexSize(Object index) { 224 | return ((Multimap) index).size(); 225 | } 226 | 227 | @SuppressWarnings("unchecked") 228 | private int getNumTuplesInWindow() { 229 | int numTuples = 0; 230 | for (Pair pairTsIndex : _indexQueue) { 231 | numTuples += ((Multimap) pairTsIndex.getRight()) 232 | .size(); 233 | } 234 | numTuples += _currMap.size(); 235 | 236 | return numTuples; 237 | } 238 | 239 | private boolean isInWindow(long tsIncoming, long tsStored) { 240 | long tsDiff = tsIncoming - tsStored; 241 | 242 | if (tsDiff >= 0) { 243 | return (tsDiff <= _thisWinSize); 244 | } 245 | else { 246 | return (-tsDiff <= _oppWinSize); 247 | } 248 | } 249 | 250 | private boolean isTimeToOutputProfile() { 251 | long currTime = _stopwatch.elapsed(SECONDS); 252 | 253 | if (currTime >= _triggerReportInSeconds) { 254 | _triggerReportInSeconds = currTime + _profileReportInSeconds; 255 | return true; 256 | } 257 | else { 258 | return false; 259 | } 260 | } 261 | 262 | private String getProfile() { 263 | StringBuilder sb = new StringBuilder(); 264 | 265 | sb.append("[Joiner-" + _rel + "-" + _tid); 266 | sb.append(" @ " + _stopwatch.elapsed(SECONDS) + " sec]"); 267 | 268 | sb.append(" #indexes=" + (_indexQueue.size() + 1)); 269 | 270 | int tuplesInWindow = getNumTuplesInWindow(); 271 | double sizeInWindow = ((double) (tuplesInWindow * _thisTupleSize)) 272 | / BYTES_PER_MB; 273 | sb.append(", tuples(window)=" + tuplesInWindow + " (" 274 | + _df.format(sizeInWindow) + " MB)"); 275 | 276 | sb.append(", tuples(stored)=" + _tuplesStored); 277 | 278 | double sizeJoined = ((double) (_tuplesStored * _oppTupleSize)) 279 | / BYTES_PER_MB; 280 | sb.append(", tuples(joined)=" + _tuplesJoined + " (" 281 | + _df.format(sizeJoined) + " MB)"); 282 | 283 | return sb.toString(); 284 | } 285 | 286 | private void output(String msg) { 287 | if (_output != null) 288 | _output.write(msg); 289 | } 290 | } 291 | 292 | -------------------------------------------------------------------------------- /src/main/java/com/basic/core/bolt/PostProcessBolt.java: -------------------------------------------------------------------------------- 1 | package com.basic.core.bolt; 2 | 3 | import java.util.*; 4 | 5 | import static java.util.concurrent.TimeUnit.MILLISECONDS; 6 | import static java.util.concurrent.TimeUnit.SECONDS; 7 | 8 | import org.apache.storm.task.OutputCollector; 9 | import org.apache.storm.task.TopologyContext; 10 | import org.apache.storm.topology.base.BaseRichBolt; 11 | import org.slf4j.Logger; 12 | import static org.slf4j.LoggerFactory.getLogger; 13 | 14 | import org.apache.storm.topology.BasicOutputCollector; 15 | import org.apache.storm.topology.OutputFieldsDeclarer; 16 | import org.apache.storm.topology.base.BaseBasicBolt; 17 | import org.apache.storm.tuple.Fields; 18 | import org.apache.storm.tuple.Tuple; 19 | import org.apache.storm.tuple.Values; 20 | import com.google.common.base.Joiner; 21 | import static com.google.common.base.Preconditions.checkArgument; 22 | import com.google.common.collect.ImmutableList; 23 | import static com.google.common.collect.Lists.newArrayList; 24 | import static com.google.common.collect.Sets.newHashSetWithExpectedSize; 25 | import com.google.common.hash.HashCode; 26 | import com.google.common.hash.HashFunction; 27 | import static com.google.common.hash.Hashing.murmur3_32; 28 | import static com.basic.core.util.CastUtils.getLong; 29 | 30 | //import backtype.storm.task.OutputCollector; 31 | //import backtype.storm.task.TopologyContext; 32 | //import backtype.storm.topology.OutputFieldsDeclarer; 33 | //import backtype.storm.topology.base.BaseRichBolt; 34 | //import backtype.storm.tuple.Fields; 35 | //import backtype.storm.tuple.Tuple; 36 | //import backtype.storm.tuple.Values; 37 | import com.basic.core.util.FileWriter; 38 | import com.basic.core.util.Stopwatch; 39 | 40 | 41 | public class PostProcessBolt extends BaseRichBolt 42 | { 43 | private static final Logger LOG = getLogger(PostProcessBolt.class); 44 | 45 | private static final List SCHEMA = ImmutableList.of("count", "sum", "min", "max"); 46 | 47 | private static final double PERCENTAGE_THRH_FOR_STANDBY = 0.5; 48 | private static final double PERCENTAGE_THRH_FOR_SWAP = 0.9; 49 | 50 | private OutputCollector _collector; 51 | 52 | private FileWriter _output; 53 | private long _currTime; 54 | // srj 55 | private long _lastOutputThroughTime; 56 | private long _lastThroughCount; 57 | private long _boltsNum; 58 | private Stopwatch _stopwatch; 59 | //map 60 | private Map> _statistics; 61 | public PostProcessBolt(int boltsNum){ 62 | _boltsNum = boltsNum; 63 | _statistics = new HashMap>(); 64 | } 65 | public void prepare(Map conf, TopologyContext context, 66 | OutputCollector collector) { 67 | _lastOutputThroughTime = System.currentTimeMillis(); 68 | _lastThroughCount = 0; 69 | 70 | _collector = collector; 71 | 72 | String prefix = "zsj_post" + context.getThisTaskId(); 73 | _output = new FileWriter("/tmp/", prefix, "csv"); 74 | _stopwatch = Stopwatch.createStarted(); 75 | _currTime = _stopwatch.elapsed(MILLISECONDS); 76 | } 77 | 78 | public void execute(Tuple tuple) { 79 | Values v = new Values(tuple.getLongByField("currentMoment"), 80 | tuple.getLongByField("tuples"), tuple.getLongByField("joinTimes"), 81 | tuple.getLongByField("processingDuration"), tuple.getDoubleByField("latency"), 82 | tuple.getLongByField("migrationTime"), tuple.getLongByField("migrationTuples"), tuple.getLongByField("resultNum")); 83 | int taskId = tuple.getSourceTask(); 84 | if(!_statistics.containsKey(taskId)){ 85 | _statistics.put(taskId, new LinkedList<>()); 86 | } 87 | _statistics.get(taskId).add(v); 88 | Long ts = getLong(_statistics.get(taskId).peek().get(0)); 89 | while(_statistics.size() == _boltsNum){ 90 | for(Map.Entry> e : _statistics.entrySet()){ 91 | Values temp = e.getValue().peek(); 92 | Long tempTs = getLong(temp.get(0)); 93 | boolean empty = false; 94 | while(tempTs.compareTo(ts) < 0){ 95 | e.getValue().poll(); 96 | if(e.getValue().isEmpty()){ 97 | _statistics.remove(e.getKey()); 98 | empty = true; 99 | break; 100 | } 101 | temp = e.getValue().peek(); 102 | tempTs = getLong(temp.get(0)); 103 | } 104 | if(empty){ 105 | break; 106 | } 107 | ts = tempTs; 108 | } 109 | if(_statistics.size() == _boltsNum) { 110 | boolean equals = true; 111 | for (Map.Entry> e : _statistics.entrySet()) { 112 | Values temp = e.getValue().peek(); 113 | Long tempTs = getLong(temp.get(0)); 114 | if(!tempTs.equals(ts)){ 115 | equals = false; 116 | break; 117 | } 118 | } 119 | long curr = _stopwatch.elapsed(MILLISECONDS); 120 | boolean hasOutput = false; 121 | if(curr - _currTime >= 1000){ 122 | if(equals){ 123 | //output statistics 124 | long tuples = 0; 125 | long joinTimes = 0; 126 | long processingDuration = 0; 127 | double latency = 0; 128 | long migrationTimes = 0; 129 | long migrationTuples = 0; 130 | LinkedList lst = new LinkedList(); 131 | int num = 0; 132 | long resultNum = 0; 133 | for (Map.Entry> e : _statistics.entrySet()) { 134 | if(!e.getValue().isEmpty()){ 135 | Values temp = e.getValue().poll(); 136 | if(e.getValue().isEmpty()){ 137 | lst.add(e.getKey()); 138 | } 139 | tuples += getLong(temp.get(1)); 140 | joinTimes += getLong(temp.get(2)); 141 | processingDuration += getLong(temp.get(3)); 142 | latency += ((Double)temp.get(4)).doubleValue(); 143 | migrationTimes += getLong(temp.get(5)); 144 | migrationTuples += getLong(temp.get(6)); 145 | resultNum += getLong(temp.get(7)); 146 | num++; 147 | } 148 | } 149 | for(Integer i : lst){ 150 | _statistics.remove(i); 151 | } 152 | StringBuilder sb = new StringBuilder(); 153 | sb.append("@ [" + ts + " sec]"); 154 | double throughput = tuples; 155 | //double throughput = resultNum; 156 | if(processingDuration == 0){ 157 | throughput = 0; 158 | } 159 | else{ 160 | //processingDuration 161 | throughput /= processingDuration; 162 | throughput *= 1000; 163 | throughput *= num; 164 | } 165 | sb.append(String.format(", %.2f", throughput)); 166 | if(tuples == 0){ 167 | latency = 0; 168 | } 169 | else{ 170 | latency /= tuples; 171 | } 172 | sb.append(String.format(", %.8f", latency)); 173 | output(sb.toString()); 174 | hasOutput = true; 175 | } 176 | } 177 | if(hasOutput){ 178 | _currTime = _stopwatch.elapsed(MILLISECONDS); 179 | } 180 | } 181 | } 182 | _collector.ack(tuple); 183 | } 184 | 185 | public void declareOutputFields(OutputFieldsDeclarer declarer) { 186 | declarer.declare(new Fields(SCHEMA)); 187 | } 188 | 189 | @Override 190 | public void cleanup() { 191 | if (_output != null) 192 | _output.endOfFile(); 193 | } 194 | private void output(String msg) { 195 | if (_output != null){ 196 | _output.writeImmediately(msg); 197 | } 198 | } 199 | 200 | 201 | private void outputSimple(String joinVal, String seq) { 202 | output(seq + " | " + joinVal); 203 | } 204 | 205 | private void outputLatency(Long tsR, Long tsS) { 206 | long tsMoreRecent = Math.max(tsR, tsS); 207 | long tsJoinResult = System.currentTimeMillis(); 208 | output("Latency: " + (tsJoinResult - tsMoreRecent) + " ms"); 209 | } 210 | 211 | } -------------------------------------------------------------------------------- /src/main/java/com/basic/core/bolt/PredictorBolt.java: -------------------------------------------------------------------------------- 1 | package com.basic.core.bolt; 2 | 3 | import com.basic.core.Constraints; 4 | import com.basic.core.inter.DumpRemoveHandler; 5 | import com.basic.core.util.PredictHeavyLoadKeyUtil; 6 | import org.apache.storm.task.OutputCollector; 7 | import org.apache.storm.task.TopologyContext; 8 | import org.apache.storm.topology.OutputFieldsDeclarer; 9 | import org.apache.storm.topology.base.BaseRichBolt; 10 | import org.apache.storm.tuple.Fields; 11 | import org.apache.storm.tuple.Tuple; 12 | import org.apache.storm.tuple.Values; 13 | import java.util.Map; 14 | 15 | /** 16 | * locate com.basic.storm.bolt 17 | * Created by windy on 2019/1/2 18 | */ 19 | public class PredictorBolt extends BaseRichBolt { 20 | private OutputCollector collector; 21 | private PredictHeavyLoadKeyUtil predict_R=PredictHeavyLoadKeyUtil.getPredictHeavyLoadKeyUtilInstance(); 22 | private PredictHeavyLoadKeyUtil predict_S=PredictHeavyLoadKeyUtil.getPredictHeavyLoadKeyUtilInstance(); 23 | public void prepare(Map conf, TopologyContext context, OutputCollector collector) { 24 | this.collector = collector; 25 | } 26 | 27 | public void execute(Tuple tuple) { 28 | String rel = tuple.getStringByField(Constraints.relFileds); 29 | final String word = tuple.getStringByField(Constraints.wordFileds); 30 | Integer count = tuple.getIntegerByField(Constraints.coinCountFileds); 31 | 32 | if (rel.equals("R")) { 33 | predict_R.PredictHeavyLoadKey(word,count); 34 | 35 | if(predict_R.isHeavyLoadKey(word)) 36 | collector.emit(new Values(rel,word,1)); 37 | predict_R.SynopsisHashMapRandomDump(new DumpRemoveHandler() { 38 | @Override 39 | public void dumpRemove(String key) { 40 | collector.emit(new Values(rel,key,0)); 41 | } 42 | }); 43 | }else{ 44 | predict_S.PredictHeavyLoadKey(word,count); 45 | 46 | if(predict_S.isHeavyLoadKey(word)) 47 | collector.emit(new Values(rel,word,1)); 48 | predict_S.SynopsisHashMapRandomDump(new DumpRemoveHandler() { 49 | @Override 50 | public void dumpRemove(String key) { 51 | collector.emit(new Values(rel,key,0)); 52 | } 53 | }); 54 | } 55 | 56 | collector.ack(tuple); 57 | } 58 | 59 | public void cleanup(){ 60 | } 61 | 62 | public void declareOutputFields(OutputFieldsDeclarer declarer) { 63 | declarer.declare(new Fields(Constraints.relFileds,Constraints.wordFileds,Constraints.typeFileds)); 64 | } 65 | } 66 | -------------------------------------------------------------------------------- /src/main/java/com/basic/core/bolt/SchedulerBolt.java: -------------------------------------------------------------------------------- 1 | package com.basic.core.bolt; 2 | 3 | import com.basic.core.Constraints; 4 | import org.apache.hadoop.util.bloom.CountingBloomFilter; 5 | import org.apache.hadoop.util.bloom.Key; 6 | import org.apache.spark.util.sketch.CountMinSketch; 7 | import org.apache.storm.task.OutputCollector; 8 | import org.apache.storm.task.TopologyContext; 9 | import org.apache.storm.topology.OutputFieldsDeclarer; 10 | import org.apache.storm.topology.base.BaseRichBolt; 11 | import org.apache.storm.tuple.Fields; 12 | import org.apache.storm.tuple.Tuple; 13 | import org.apache.storm.tuple.Values; 14 | 15 | import java.util.Map; 16 | 17 | /** 18 | * locate com.basic.storm.bolt 19 | * Created by windy on 2019/1/2 20 | */ 21 | public class SchedulerBolt extends BaseRichBolt { 22 | private String UPSTREAM_COMPONENT_ID; 23 | private String UPSTREAM_FIELDS; 24 | private OutputCollector collector; 25 | private CountingBloomFilter bf; 26 | private CountingMinSketch cms; 27 | private static final List SCHEMA = ImmutableList.of("relation", "timestamp", "key", "value"); 28 | 29 | public SchedulerBolt(String UPSTREAM_COMPONENT_ID,String UPSTREAM_REL_FIELDS, UPSTREAM_KEY_FIELDS) { 30 | this.UPSTREAM_COMPONENT_ID = UPSTREAM_COMPONENT_ID; 31 | this.UPSTREAM_FIELDS=UPSTREAM_FIELDS; 32 | } 33 | 34 | public void prepare(Map conf, TopologyContext context, OutputCollector collector) { 35 | this.collector = collector; 36 | this.bf_R = new CountingBloomFilter(16,4,1); 37 | this.bf_S = new CountingBloomFilter(16,4,1); 38 | this.cms_R = CountMinSketch.create(0.05, 0.05, 5); 39 | this.cms_S = CountMinSketch.create(0.05, 0.05, 5); 40 | } 41 | 42 | public void execute(Tuple tuple) { 43 | if(tuple.getSourceComponent().equals(UPSTREAM_COMPONENT_ID)){ 44 | String rel = tuple.getStringByField(UPSTREAM_REL_FIELDS); 45 | String word = tuple.getStringByField(UPSTREAM_KEY_FIELDS); 46 | 47 | Long ts = tuple.getStringByField("Constraints.tsFileds"); 48 | String value = tuple.getStringByField("Constraints.valueSFileds"); 49 | 50 | if(word.length() <= 0) { 51 | collector.ack(tuple); 52 | return; 53 | } 54 | 55 | Key ky = new Key(word.getBytes()); 56 | 57 | if (rel.equals("R")) { 58 | cms_R.add(ky); 59 | Long baseCount = cms_R.estimateCount(ky); 60 | collector.emit(Constraints.coinFileds,new Values(rel,word,baseCount)); 61 | if(bf_R.membershipTest(ky)) 62 | collector.emit(Constraints.hotRFileds, tuple, new Values(rel,ts,word,value)); 63 | else 64 | collector.emit(Constraints.nohotRFileds, tuple, new Values(rel,ts,word,value)); 65 | }else{ 66 | cms_S.add(ky); 67 | Long baseCount = cms_S.estimateCount(ky); 68 | collector.emit(Constraints.coinFileds,new Values(rel,word,baseCount)); 69 | if(bf_S.membershipTest(ky)) 70 | collector.emit(Constraints.hotSFileds, tuple, new Values(rel,ts,word,value)); 71 | else 72 | collector.emit(Constraints.nohotSFileds, tuple, new Values(rel,ts,word,value)); 73 | } 74 | 75 | }else { 76 | String rel = tuple.getStringByField(Constraints.relFileds); 77 | String key = tuple.getStringByField(Constraints.wordFileds); 78 | Integer type = tuple.getIntegerByField(Constraints.typeFileds); 79 | Key hk = new Key(key.getBytes()); 80 | if (rel.equals("R")) { 81 | if(!bf_R.membershipTest(hk) && type.equals(1)) 82 | bf_R.add(hk); 83 | if(bf_R.membershipTest(hk) && type.equals(0)) 84 | bf_R.delete(hk); 85 | }else{ 86 | if(!bf_S.membershipTest(hk) && type.equals(1)) 87 | bf_S.add(hk); 88 | if(bf.membershipTest(hk) && type.equals(0)) 89 | bf_S.delete(hk); 90 | } 91 | } 92 | collector.ack(tuple); 93 | } 94 | 95 | public void cleanup(){ 96 | } 97 | 98 | public void declareOutputFields(OutputFieldsDeclarer declarer) { 99 | declarer.declareStream(Constraints.coinFileds, new Fields(Constraints.relFileds,Constraints.wordFileds,Constraints.coinCountFileds)); 100 | declarer.declareStream(Constraints.hotRFileds, new Fields(SCHEMA)); 101 | declarer.declareStream(Constraints.nohotRFileds, new Fields(SCHEMA)); 102 | declarer.declareStream(Constraints.hotSFileds, new Fields(SCHEMA)); 103 | declarer.declareStream(Constraints.nohotSFileds, new Fields(SCHEMA)); 104 | } 105 | 106 | } 107 | -------------------------------------------------------------------------------- /src/main/java/com/basic/core/bolt/ShuffleBolt.java: -------------------------------------------------------------------------------- 1 | package com.basic.core.bolt; 2 | 3 | import java.util.List; 4 | import java.util.Map; 5 | 6 | import com.google.common.collect.ImmutableList; 7 | import static com.google.common.collect.Lists.newArrayList; 8 | 9 | import org.apache.storm.task.OutputCollector; 10 | import org.apache.storm.task.TopologyContext; 11 | import org.apache.storm.topology.BasicOutputCollector; 12 | import org.apache.storm.topology.OutputFieldsDeclarer; 13 | import org.apache.storm.topology.base.BaseBasicBolt; 14 | import org.apache.storm.topology.base.BaseRichBolt; 15 | import org.apache.storm.tuple.Fields; 16 | import org.apache.storm.tuple.Tuple; 17 | import org.apache.storm.tuple.Values; 18 | import org.apache.storm.Constants; 19 | import com.basic.core.util.FileWriter; 20 | import com.basic.core.util.GeoHash; 21 | import com.basic.core.util.Stopwatch; 22 | 23 | import static java.util.concurrent.TimeUnit.MILLISECONDS; 24 | import static java.util.concurrent.TimeUnit.SECONDS; 25 | import static com.basic.core.util.CastUtils.getList; 26 | 27 | public class 28 | ShuffleBolt extends BaseRichBolt 29 | { 30 | private OutputCollector _collector; 31 | private static final List SCHEMA = ImmutableList.of("relation", "timestamp", "key", "value"); 32 | private FileWriter _output; 33 | private long _r; 34 | private long _s; 35 | private long _lastTime; 36 | private Stopwatch _stopwatch; 37 | private String _rStream; 38 | private String _sStream; 39 | public ShuffleBolt(String datasize){ 40 | _rStream = "didiOrder" + datasize; 41 | _sStream = "didiGps" + datasize; 42 | } 43 | @Override 44 | public void prepare(Map map, TopologyContext topologyContext, OutputCollector outputCollector) { 45 | this._collector = outputCollector; 46 | _output = new FileWriter("/tmp/", "zsj_shuffle" + topologyContext.getThisTaskId(), "txt"); 47 | _r = 0; 48 | _s = 0; 49 | _lastTime = 0; 50 | _stopwatch = Stopwatch.createStarted(); 51 | 52 | } 53 | 54 | public void declareOutputFields(OutputFieldsDeclarer declarer) { 55 | declarer.declare(new Fields(SCHEMA)); 56 | } 57 | @Override 58 | public void execute(Tuple tuple) { 59 | String topic = tuple.getStringByField("topic"); 60 | String value = tuple.getStringByField("value"); 61 | String rel; 62 | Long ts = System.currentTimeMillis(); 63 | String []cols = value.split(","); 64 | //output(value); 65 | String key = ""; 66 | if(topic.equals(_rStream)) { 67 | rel = "R"; 68 | //cols[1] is order id 69 | key = GeoHash.encode(Double.parseDouble(cols[4]), Double.parseDouble(cols[3]),7).toHashString(); 70 | _r ++; 71 | } else if(topic.equals(_sStream)) { 72 | rel = "S"; 73 | //cols[0] is order id 74 | key = GeoHash.encode(Double.parseDouble(cols[4]), Double.parseDouble(cols[3]),7).toHashString(); 75 | _s++; 76 | } else { 77 | rel = "false"; 78 | } 79 | _collector.emit(new Values(rel, ts, key, value)); 80 | } 81 | 82 | private void output(String msg) { 83 | if (_output != null){ 84 | //_output.write(msg); 85 | _output.writeImmediately(msg); 86 | } 87 | 88 | } 89 | @Override 90 | public void cleanup() { 91 | if (_output != null) { 92 | _output.endOfFile(); 93 | } 94 | } 95 | } 96 | -------------------------------------------------------------------------------- /src/main/java/com/basic/core/inter/BinaryComparable.java: -------------------------------------------------------------------------------- 1 | package com.basic.core.inter; 2 | 3 | import org.apache.hadoop.io.WritableComparator; 4 | 5 | /** 6 | * locate com.basic.storm.inter 7 | * Created by tj on 2017/5/9. 8 | */ 9 | public abstract class BinaryComparable implements Comparable { 10 | public BinaryComparable() { 11 | } 12 | 13 | public abstract int getLength(); 14 | 15 | public abstract byte[] getBytes(); 16 | 17 | public int compareTo(BinaryComparable other) { 18 | return this == other?0: WritableComparator.compareBytes(this.getBytes(), 0, this.getLength(), other.getBytes(), 0, other.getLength()); 19 | } 20 | 21 | public int compareTo(byte[] other, int off, int len) { 22 | return WritableComparator.compareBytes(this.getBytes(), 0, this.getLength(), other, off, len); 23 | } 24 | 25 | public boolean equals(Object other) { 26 | if(!(other instanceof BinaryComparable)) { 27 | return false; 28 | } else { 29 | BinaryComparable that = (BinaryComparable)other; 30 | return this.getLength() != that.getLength()?false:this.compareTo(that) == 0; 31 | } 32 | } 33 | 34 | public int hashCode() { 35 | return WritableComparator.hashBytes(this.getBytes(), this.getLength()); 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /src/main/java/com/basic/core/inter/DumpRemoveHandler.java: -------------------------------------------------------------------------------- 1 | package com.basic.core.inter; 2 | 3 | /** 4 | * locate com.basic.storm.util 5 | * Created by tj on 2017/5/8. 6 | */ 7 | public interface DumpRemoveHandler { 8 | public void dumpRemove(String key); 9 | } 10 | -------------------------------------------------------------------------------- /src/main/java/com/basic/core/util/CastUtils.java: -------------------------------------------------------------------------------- 1 | package com.basic.core.util; 2 | 3 | import java.util.List; 4 | 5 | public class CastUtils 6 | { 7 | public static String getString(Object o) { 8 | if (o == null) { 9 | return null; 10 | } 11 | else { 12 | return o.toString(); 13 | } 14 | } 15 | 16 | public static String getString(List values, int i) { 17 | return getString(values.get(i)); 18 | } 19 | 20 | public static int getInt(Object o) { 21 | if (o instanceof Long) { 22 | return ((Long) o).intValue(); 23 | } 24 | else if (o instanceof Integer) { 25 | return ((Integer) o).intValue(); 26 | } 27 | else if (o instanceof Short) { 28 | return ((Short) o).intValue(); 29 | } 30 | else if (o instanceof String) { 31 | return Integer.parseInt((String) o); 32 | } 33 | else { 34 | throw new IllegalArgumentException("Failed to convert " + o 35 | + " to int"); 36 | } 37 | } 38 | 39 | public static int getInt(List values, int i) { 40 | return getInt(values.get(i)); 41 | } 42 | 43 | public static long getLong(Object o) { 44 | if (o instanceof Long) { 45 | return ((Long) o).longValue(); 46 | } 47 | else if (o instanceof Integer) { 48 | return ((Integer) o).longValue(); 49 | } 50 | else if (o instanceof Short) { 51 | return ((Short) o).longValue(); 52 | } 53 | else if (o instanceof String) { 54 | return Long.parseLong((String) o); 55 | } 56 | else { 57 | throw new IllegalArgumentException("Failed to convert " + o 58 | + " to long"); 59 | } 60 | } 61 | 62 | public static long getLong(List values, int i) { 63 | return getLong(values.get(i)); 64 | } 65 | 66 | public static boolean getBoolean(Object o) { 67 | if (o instanceof Boolean) { 68 | return ((Boolean) o).booleanValue(); 69 | } 70 | else if (o instanceof String) { 71 | return Boolean.parseBoolean((String) o); 72 | } 73 | else { 74 | throw new IllegalArgumentException("Failed to convert " + o 75 | + " to boolean"); 76 | } 77 | } 78 | 79 | public static boolean getBoolean(List values, int i) { 80 | return getBoolean(values.get(i)); 81 | } 82 | 83 | public static List getList(Object o) { 84 | if (o instanceof List) { 85 | return (List) o; 86 | } 87 | else { 88 | throw new IllegalArgumentException("Failed to convert " + o 89 | + " to list of objects"); 90 | } 91 | } 92 | } -------------------------------------------------------------------------------- /src/main/java/com/basic/core/util/FileReader.java: -------------------------------------------------------------------------------- 1 | package com.basic.core.util; 2 | 3 | import java.io.BufferedReader; 4 | import java.io.File; 5 | import java.io.IOException; 6 | import java.util.List; 7 | import java.util.regex.Pattern; 8 | 9 | //import static org.apache.commons.io.FilenameUtils.normalize; 10 | import org.slf4j.Logger; 11 | 12 | import static org.apache.commons.io.FilenameUtils.normalize; 13 | import static org.slf4j.LoggerFactory.getLogger; 14 | 15 | import com.google.common.base.Splitter; 16 | import static com.google.common.base.Charsets.UTF_8; 17 | import com.google.common.collect.ImmutableList; 18 | import static com.google.common.collect.Lists.newArrayList; 19 | import static com.google.common.io.Files.newReader; 20 | 21 | public class FileReader 22 | { 23 | private static final String DEFAULT_SPLIT_PATTERN = "\\s+"; 24 | private static final Logger LOG = getLogger(FileReader.class); 25 | 26 | private final String _filename; 27 | private final BufferedReader _reader; 28 | private final Splitter _splitter; 29 | 30 | public FileReader(String filename, String splitPunc) throws Exception { 31 | _filename = normalize(filename); 32 | _reader = getReader(); 33 | 34 | String splitePattern; 35 | if (splitPunc == null || splitPunc.trim().isEmpty()) { 36 | splitePattern = DEFAULT_SPLIT_PATTERN; 37 | } 38 | else { 39 | splitePattern = splitPunc.trim() + "\\s+"; 40 | } 41 | _splitter = Splitter.on(Pattern.compile(splitePattern)).trimResults(); 42 | } 43 | 44 | private BufferedReader getReader() throws Exception { 45 | if (_filename == null || _filename.isEmpty()) { 46 | LOG.debug("No input file is specified"); 47 | return null; 48 | } 49 | 50 | File file = new File(_filename); 51 | if (!file.exists()) { 52 | LOG.error("File not exists: " + _filename); 53 | return null; 54 | } 55 | else { 56 | return newReader(file, UTF_8); 57 | } 58 | } 59 | 60 | public String readLine() throws IOException { 61 | if (_reader == null) { 62 | return null; 63 | } 64 | else { 65 | return _reader.readLine(); 66 | } 67 | } 68 | 69 | public List readLineAndSplit() throws IOException { 70 | String line = readLine(); 71 | 72 | if (line == null) 73 | return null; 74 | else 75 | return newArrayList(_splitter.split(line)); 76 | } 77 | 78 | public String getFilename() { 79 | return _filename; 80 | } 81 | } -------------------------------------------------------------------------------- /src/main/java/com/basic/core/util/FileWriter.java: -------------------------------------------------------------------------------- 1 | package com.basic.core.util; 2 | 3 | import java.io.BufferedWriter; 4 | import java.io.File; 5 | import java.io.FileNotFoundException; 6 | import java.io.IOException; 7 | import java.io.PrintStream; 8 | 9 | //import static org.apache.commons.io.FilenameUtils.getFullPath; 10 | //import static org.apache.commons.io.FilenameUtils.normalize; 11 | import org.slf4j.Logger; 12 | 13 | import static org.apache.commons.io.FilenameUtils.getFullPath; 14 | import static org.apache.commons.io.FilenameUtils.normalize; 15 | import static org.slf4j.LoggerFactory.getLogger; 16 | 17 | import static com.google.common.base.Charsets.UTF_8; 18 | import static com.google.common.io.Files.newWriter; 19 | 20 | public class FileWriter 21 | { 22 | private static final int DEFAULT_FLUSH_SIZE = 512; 23 | private static final Logger LOG = getLogger(FileWriter.class); 24 | 25 | private final String _filename; 26 | private final BufferedWriter _writer; 27 | 28 | private PrintStream _printer = null; 29 | 30 | private int _flushSize = DEFAULT_FLUSH_SIZE; 31 | private int _count = 0; 32 | 33 | public FileWriter(PrintStream ps) { 34 | _filename = null; 35 | _writer = null; 36 | _printer = ps; 37 | } 38 | 39 | public FileWriter(String filename, boolean overwrite) { 40 | _filename = normalize(filename); 41 | _writer = getWriter(overwrite); 42 | } 43 | 44 | public FileWriter(String filename) { 45 | _filename = normalize(filename); 46 | _writer = getWriter(false); 47 | } 48 | 49 | public FileWriter(String dir, String prefix, String suffix, 50 | boolean overwrite) { 51 | _filename = constructFilename(dir, prefix, suffix); 52 | _writer = getWriter(overwrite); 53 | } 54 | 55 | public FileWriter(String dir, String prefix, String suffix) { 56 | _filename = constructFilename(dir, prefix, suffix); 57 | _writer = getWriter(false); 58 | } 59 | 60 | private String constructFilename(String dir, String prefix, String suffix) { 61 | String filename; 62 | 63 | if (dir == null) { 64 | filename = null; 65 | } 66 | else { 67 | filename = (dir.isEmpty() ? "." : dir) + "/"; 68 | 69 | if (prefix == null || prefix.isEmpty()) { 70 | filename += ""; 71 | } 72 | else { 73 | filename += prefix + "_"; 74 | } 75 | filename += TimeUtils.getTimestamp(); 76 | 77 | filename += "." + suffix; 78 | } 79 | 80 | return normalize(filename); 81 | } 82 | 83 | private BufferedWriter getWriter(boolean overwrite) { 84 | if (_filename == null || _filename.isEmpty()) { 85 | LOG.debug("Dummy file writer is in use"); 86 | return null; 87 | } 88 | 89 | mkdir(getFullPath(_filename)); 90 | 91 | BufferedWriter writer; 92 | File file = new File(_filename); 93 | if (file.exists()) { 94 | if (overwrite) { 95 | file.delete(); 96 | createNewFile(file); 97 | LOG.info("Overwriting existing file: " + _filename); 98 | writer = getWriter(file); 99 | } 100 | else { 101 | LOG.error("File exists (not overwriting): " + _filename); 102 | writer = null; 103 | } 104 | } 105 | else { 106 | createNewFile(file); 107 | LOG.debug("New file created: " + _filename); 108 | writer = getWriter(file); 109 | } 110 | 111 | return writer; 112 | } 113 | 114 | private BufferedWriter getWriter(File file) { 115 | BufferedWriter writer = null; 116 | try { 117 | writer = newWriter(file, UTF_8); 118 | } 119 | catch (FileNotFoundException e) { 120 | LOG.error(e.getMessage()); 121 | } 122 | 123 | return writer; 124 | } 125 | 126 | private void mkdir(String path) { 127 | File dir = new File(path); 128 | dir.mkdirs(); 129 | } 130 | 131 | private void createNewFile(File file) { 132 | try { 133 | file.createNewFile(); 134 | } 135 | catch (IOException e) { 136 | LOG.error(e.getMessage()); 137 | } 138 | } 139 | 140 | public void write(String msg) { 141 | append(msg); 142 | flush(false); 143 | 144 | if (_printer != null) { 145 | _printer.println(msg); 146 | } 147 | } 148 | 149 | public void endOfFile() { 150 | write("EOF"); 151 | flush(true); 152 | } 153 | 154 | private void append(String msg) { 155 | if (_writer != null) { 156 | try { 157 | _writer.append(msg + "\n"); 158 | } 159 | catch (IOException e) { 160 | LOG.error(e.getMessage()); 161 | } 162 | 163 | ++_count; 164 | } 165 | } 166 | 167 | private void flush(boolean immediate) { 168 | if (_writer != null) { 169 | if (immediate || (_count % _flushSize == 0)) { 170 | try { 171 | _writer.flush(); 172 | } 173 | catch (IOException e) { 174 | LOG.error(e.getMessage()); 175 | } 176 | } 177 | } 178 | } 179 | 180 | public String getFilename() { 181 | return _filename; 182 | } 183 | 184 | public boolean isNull() { 185 | return (_writer == null); 186 | } 187 | 188 | public FileWriter setFlushSize(int size) { 189 | _flushSize = size; 190 | return this; 191 | } 192 | 193 | public FileWriter setPrintStream(PrintStream ps) { 194 | _printer = ps; 195 | return this; 196 | } 197 | } -------------------------------------------------------------------------------- /src/main/java/com/basic/core/util/GeoHash.java: -------------------------------------------------------------------------------- 1 | package com.basic.core.util; 2 | 3 | import org.apache.commons.lang3.tuple.ImmutablePair; 4 | import org.apache.commons.lang3.tuple.Pair; 5 | 6 | import java.math.BigDecimal; 7 | import java.util.Arrays; 8 | import java.util.*; 9 | public class GeoHash { 10 | 11 | protected static final byte[] characters = { 12 | // 0 1 2 3 4 5 6 7 13 | '0', '1', '2', '3', '4', '5', '6', '7', 14 | // 8 9 10 11 12 13 14 15 15 | '8', '9', 'b', 'c', 'd', 'e', 'f', 'g', 16 | // 16 17 18 19 20 21 22 23 17 | 'h', 'j', 'k', 'm', 'n', 'p', 'q', 'r', 18 | // 24 25 26 27 28 29 30 31 19 | 's', 't', 'u', 'v', 'w', 'x', 'y', 'z' }; 20 | protected static final byte[] map = new byte['z' + 1]; 21 | static { 22 | for (byte i = 0; i < characters.length; i++) 23 | map[characters[i]] = i; 24 | } 25 | 26 | /** number of bits per character */ 27 | protected static final int BITS_PER_CHARACTER = 5; 28 | protected static final int MAX_BITS = 60; 29 | protected static final double[] LATITUDE_RANGE = { -90.0, 90.0 }; 30 | protected static final double[] LONGITUDE_RANGE = { -180.0, 180.0 }; 31 | 32 | public final double lat, lon; 33 | /** 34 | * The precision of the coordinate. Must be in the interval [1-12] 35 | */ 36 | public final int precision; 37 | /** 38 | * The bit representation of the (LAT,LON)-pair 39 | */ 40 | public final long bitValue; 41 | /** 42 | * The encoded representation of the coordinate 43 | */ 44 | public final byte[] hash; 45 | 46 | protected GeoHash(final double lat, final double lon, final long bitValue, final byte[] hash) { 47 | this.lat = lat; 48 | this.lon = lon; 49 | this.precision = hash.length; 50 | this.bitValue = bitValue; 51 | this.hash = hash; 52 | } 53 | 54 | /** 55 | * @return The encoded geohash 56 | */ 57 | public final String toHashString() { 58 | return new String(hash); 59 | } 60 | 61 | /** 62 | * @return The binary representation of the coordinate 63 | */ 64 | public final String toBinaryRepresentation() { 65 | return new String(binaryRepresentation(bitValue, precision)); 66 | } 67 | 68 | public static final GeoHash decode(final String hash) { 69 | return decode(hash.getBytes()); 70 | } 71 | 72 | public static final GeoHash decode(final byte[] hash) { 73 | int lat = 0, lon = 0;// this gives us a bit length of 32 for each coordinate - ought to be sufficient 74 | boolean evenbit = true; 75 | 76 | // split hash into binary latitude and longitude parts 77 | long binary = 0; 78 | for (byte b : hash) { 79 | b = (byte) (0x1F & map[b]); 80 | binary <<= BITS_PER_CHARACTER; 81 | binary |= b; 82 | // unrolled loop over each bit 83 | if (evenbit) { 84 | lon = extractEvenBits(lon, b); 85 | lat = extractUnevenBits(lat, b); 86 | } else { 87 | lat = extractEvenBits(lat, b); 88 | lon = extractUnevenBits(lon, b); 89 | } 90 | evenbit = !evenbit; 91 | } 92 | 93 | final double latitude = decodeCoordinate( lat, new GeoHash.Coordinate(0.0, LATITUDE_RANGE, calculateLatitudeBits(hash.length))).coord; 94 | final double longitude = decodeCoordinate( lon, new GeoHash.Coordinate(0.0, LONGITUDE_RANGE, calculateLongitudeBits(hash.length))).coord; 95 | 96 | return new GeoHash( latitude , longitude, binary, hash); 97 | } 98 | 99 | protected static final int calculateLatitudeBits(final int precision) { 100 | return calculateBits(precision, 2); 101 | } 102 | 103 | protected static final int calculateLongitudeBits(final int precision) { 104 | return calculateBits(precision, 3); 105 | } 106 | 107 | /** 108 | * A mathematical way of calculating the number of bits from input length 109 | * (length / 2 * 5) + (length % 2 != 0 ? 3 : 0) 110 | */ 111 | private static final int calculateBits(final int precision, final int unevenExtra) { 112 | return (((precision >> 1) * BITS_PER_CHARACTER) + ((precision & 0x1) * unevenExtra)); 113 | } 114 | 115 | /** 116 | * Extracts the even bits, starting with index 1 117 | * Example: 118 | * 00010110 = 0x16 119 | * | | | | 120 | * 0 1 1 0 = 0x06 121 | */ 122 | protected static final int extractEvenBits(int value, final byte b) { 123 | value <<= 3; 124 | value |= ((b & 0x10) >> 2); 125 | value |= ((b & 0x04) >> 1); 126 | value |= (b & 0x01); 127 | return value; 128 | } 129 | 130 | /** 131 | * Extracts the uneven bits, starting with index 0 132 | * Example: 133 | * 00010110 = 0x16 134 | * | | | | 135 | * 0 0 0 1 = 0x01 136 | */ 137 | protected static final int extractUnevenBits(int value, final byte b) { 138 | value <<= 2; 139 | value |= ((b & 0x08) >> 2); 140 | value |= ((b & 0x02) >> 1); 141 | return value; 142 | } 143 | 144 | protected static final Coordinate decodeCoordinate(final long bitCoord, final Coordinate coord) { 145 | double val = 0.0; 146 | int mask = 1 << coord.bits; 147 | while ((mask >>= 1) >= 1) { // while bits are left to be explored 148 | if ((mask & bitCoord) > 0) {// bit == 1 149 | coord.min = val; 150 | val = (val + coord.max) / 2; 151 | } else { // bit == 0 152 | coord.max = val; 153 | val = (val + coord.min) / 2; 154 | } 155 | } 156 | // some rounding might be needed 157 | coord.coord = new BigDecimal(val).setScale(coord.bits / 5, BigDecimal.ROUND_HALF_UP).doubleValue(); 158 | return coord; 159 | } 160 | 161 | /** 162 | * Encodes the coordinate-pair into the hash representation 163 | * 164 | * @param lat 165 | * Latitude coordinate 166 | * @param lon 167 | * Longitude coordinate 168 | * @param precision 169 | * Geohash length must be in the interval [1-12] 170 | * @return the GeoHash object holding information about the coordinates and 171 | * hashed values 172 | */ 173 | public static final GeoHash encode(final double lat, final double lon, int precision) { 174 | if (precision < 1) precision = 1; 175 | 176 | final Coordinate latInfo = new Coordinate(lat, LATITUDE_RANGE, calculateLatitudeBits(precision)); 177 | final Coordinate lonInfo = new Coordinate(lon, LONGITUDE_RANGE, calculateLongitudeBits(precision)); 178 | 179 | // precision cannot be more than 60 bits (the nearest multiple of 5 under 64 (the bits of a long)) 180 | long mask = 0x1l << Math.min(precision * BITS_PER_CHARACTER, MAX_BITS); 181 | long bitValue = 0; 182 | boolean even = true; 183 | while ((mask >>= 1) > 0) { 184 | if (even) { 185 | // longitude 186 | bitValue = encode(bitValue, lonInfo); 187 | } else { 188 | // latitude 189 | bitValue = encode(bitValue, latInfo); 190 | } 191 | even = !even; 192 | } 193 | 194 | return new GeoHash(lat, lon, bitValue, translateBinaryToHash(bitValue, precision)); 195 | } 196 | 197 | /** 198 | * See {@link GeoHash#encode(double, double, int)}. 199 | * This method defaults to the maximal value of precision 200 | */ 201 | public static final GeoHash encode(final double lat, final double lon) { 202 | return encode(lat, lon, MAX_BITS / BITS_PER_CHARACTER); 203 | } 204 | 205 | protected static final long encode(long bitValue, final Coordinate info) { 206 | info.mid = (info.min + info.max) / 2; 207 | if (info.coord >= info.mid) { 208 | bitValue <<= 1; 209 | bitValue |= 0x1;// add one 210 | info.min = info.mid; 211 | } else { 212 | bitValue <<= 1;// add zero 213 | info.max = info.mid; 214 | } 215 | return bitValue; 216 | } 217 | 218 | protected static final byte[] translateBinaryToHash(long value, int precision) { 219 | final byte[] h = new byte[precision]; 220 | while (precision > 0) { 221 | h[--precision] = characters[(byte) (value & 0x1F)]; 222 | value >>= BITS_PER_CHARACTER; 223 | } 224 | return h; 225 | } 226 | 227 | protected static final String binaryRepresentation(final long bitValue, final int precision) { 228 | char[] rep = new char[Math.min(precision * BITS_PER_CHARACTER, MAX_BITS)]; 229 | int index = 0; 230 | long mask = 0x1l << rep.length; 231 | while ((mask >>= 1) >= 1) // while bits are left to be explored 232 | rep[index++] = ((mask & bitValue) > 0 ? '1' : '0'); 233 | 234 | return new String(rep); 235 | } 236 | 237 | @Override 238 | public String toString() { 239 | return String.format("%f %f %d %s", this.lat, this.lon, this.bitValue, binaryRepresentation(this.bitValue, this.precision)); 240 | } 241 | 242 | @Override 243 | public boolean equals(Object obj) { 244 | if (obj != null && obj instanceof GeoHash) 245 | return Arrays.equals(((GeoHash) obj).hash, this.hash); 246 | return false; 247 | } 248 | 249 | @Override 250 | public int hashCode() { 251 | return new String(hash).hashCode(); 252 | } 253 | 254 | /** 255 | * Dataholder for a single coordinate 256 | */ 257 | protected static final class Coordinate { 258 | protected double coord, min, max, mid; 259 | protected int bits; 260 | 261 | public Coordinate(final double coordinate, final double[] range, final int bits) { 262 | this.coord = coordinate; 263 | this.min = range[0]; 264 | this.max = range[1]; 265 | this.mid = 0.0; 266 | this.bits = bits; 267 | } 268 | } 269 | 270 | static class A { 271 | private Map _map; 272 | public A(Map _map){ 273 | this._map = _map; 274 | } 275 | public String toString(){ 276 | StringBuffer sb = new StringBuffer(); 277 | sb.append("Object: \n"); 278 | for(Map.Entry e : this._map.entrySet()){ 279 | sb.append(e); 280 | sb.append("\n"); 281 | } 282 | return sb.toString(); 283 | } 284 | } 285 | public static void main(String []argc){ 286 | System.out.println(GeoHash.encode(30.703971, 104.094640, 7).toHashString()); 287 | PriorityQueue pq = new PriorityQueue(new Comparator() { 288 | @Override 289 | public int compare(Pair o1, Pair o2) { 290 | if((Long)o1.getLeft() > (Long)o2.getLeft()){ 291 | return -1; 292 | } 293 | else if((Long)o1.getLeft() < (Long)o2.getLeft()){ 294 | return 1; 295 | } 296 | return 0; 297 | } 298 | }); 299 | pq.add(ImmutablePair.of(102L, "abc")); 300 | pq.add(ImmutablePair.of(122L, "abc")); 301 | pq.add(ImmutablePair.of(322L, "abc")); 302 | while(!pq.isEmpty()){ 303 | System.out.println(pq.poll().getLeft()); 304 | } 305 | 306 | LinkedList lst = new LinkedList(); 307 | lst.add(1);lst.add(2);lst.add(3);lst.add(5);lst.add(1);lst.add(2);lst.add(3);lst.add(5); 308 | for(Integer i : lst){ 309 | if(i.equals(1)){ 310 | lst.remove(i); 311 | } 312 | } 313 | for(Integer i : lst){ 314 | System.out.print(i + " "); 315 | } 316 | } 317 | } -------------------------------------------------------------------------------- /src/main/java/com/basic/core/util/HdfsOperationUtil.java: -------------------------------------------------------------------------------- 1 | package com.basic.core.util; 2 | 3 | import org.apache.hadoop.conf.Configuration; 4 | import org.apache.hadoop.fs.FileSystem; 5 | import org.apache.hadoop.hdfs.DistributedFileSystem; 6 | 7 | /** 8 | * Created by windy on 2019/1/2 9 | * HDFSOperatorUtil 10 | */ 11 | public class HdfsOperationUtil { 12 | 13 | private static Configuration conf = new Configuration(); 14 | private static final String HADOOP_URL="hdfs://192.168.223.202:9000";// default HDFS configuration 15 | 16 | private static FileSystem fs; 17 | 18 | private static DistributedFileSystem hdfs; 19 | 20 | static { 21 | try { 22 | FileSystem.setDefaultUri(conf, HADOOP_URL); 23 | fs = FileSystem.get(conf); 24 | hdfs = (DistributedFileSystem)fs; 25 | } catch (Exception e) { 26 | e.printStackTrace(); 27 | } 28 | } 29 | 30 | public static FileSystem getFs() { 31 | return fs; 32 | } 33 | 34 | public static Configuration getConf() { 35 | return conf; 36 | } 37 | 38 | public static void setConf(Configuration conf) { 39 | HdfsOperationUtil.conf = conf; 40 | } 41 | 42 | } 43 | 44 | -------------------------------------------------------------------------------- /src/main/java/com/basic/core/util/LogHelpers.java: -------------------------------------------------------------------------------- 1 | package com.basic.core.util; 2 | 3 | import java.util.Map; 4 | 5 | import org.apache.storm.generated.Bolt; 6 | import org.apache.storm.generated.ComponentCommon; 7 | import org.apache.storm.generated.SpoutSpec; 8 | import org.apache.storm.generated.StormTopology; 9 | import org.slf4j.Logger; 10 | 11 | //import backtype.storm.generated.Bolt; 12 | //import backtype.storm.generated.ComponentCommon; 13 | //import backtype.storm.generated.SpoutSpec; 14 | //import backtype.storm.generated.StormTopology; 15 | 16 | public class LogHelpers 17 | { 18 | public static void logTopology(Logger logger, StormTopology t) { 19 | logger.info("number of spouts: " + t.get_spouts_size()); 20 | 21 | for (Map.Entry kv : t.get_spouts().entrySet()) { 22 | logger.info("[Spout] " + kv.getKey() + ", " 23 | + getString(kv.getValue().get_common())); 24 | } 25 | 26 | logger.info("number of bolts: " + t.get_bolts_size()); 27 | 28 | for (Map.Entry kv : t.get_bolts().entrySet()) { 29 | logger.info("[Bolt] " + kv.getKey() + ", " 30 | + getString(kv.getValue().get_common())); 31 | } 32 | } 33 | 34 | private static String getString(ComponentCommon common) { 35 | StringBuilder sb = new StringBuilder(); 36 | 37 | sb.append("parallelism:" + common.get_parallelism_hint()); 38 | sb.append(", inputs:" + common.get_inputs()); 39 | sb.append(", streams:" + common.get_streams()); 40 | 41 | return sb.toString(); 42 | } 43 | } -------------------------------------------------------------------------------- /src/main/java/com/basic/core/util/MyScheme.java: -------------------------------------------------------------------------------- 1 | package com.basic.core.util; 2 | 3 | import org.apache.storm.spout.Scheme; 4 | import org.apache.storm.tuple.Fields; 5 | import org.apache.storm.tuple.Values; 6 | import org.apache.storm.utils.Utils; 7 | 8 | import java.nio.ByteBuffer; 9 | import java.nio.charset.Charset; 10 | import java.nio.charset.StandardCharsets; 11 | import java.util.List; 12 | 13 | /** 14 | * locate com.basic.core.util 15 | * Created by windy on 2019/1/2 16 | */ 17 | public class MyScheme implements Scheme { 18 | private static final Charset UTF8_CHARSET = StandardCharsets.UTF_8; 19 | public static String STRING_SCHEME_KEY = "word"; 20 | 21 | public List deserialize(ByteBuffer bytes) { 22 | return new Values(deserializeString(bytes)); 23 | } 24 | 25 | public static String deserializeString(ByteBuffer string) { 26 | if (string.hasArray()) { 27 | int base = string.arrayOffset(); 28 | return new String(string.array(), base + string.position(), string.remaining()); 29 | } else { 30 | return new String(Utils.toByteArray(string), UTF8_CHARSET); 31 | } 32 | } 33 | 34 | public Fields getOutputFields() { 35 | return new Fields(STRING_SCHEME_KEY); 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /src/main/java/com/basic/core/util/PredictHeavyLoadKeyUtil.java: -------------------------------------------------------------------------------- 1 | package com.basic.core.util; 2 | 3 | import com.basic.core.inter.DumpRemoveHandler; 4 | 5 | import java.io.Serializable; 6 | import java.util.BitSet; 7 | import java.util.Iterator; 8 | import java.util.Map; 9 | import java.util.Random; 10 | 11 | import static com.basic.core.Constraints.*; 12 | 13 | /** 14 | * locate com.basic.storm.util 15 | * Created by windy on 2019/1/2. 16 | */ 17 | public class PredictHeavyLoadKeyUtil implements Serializable{ 18 | 19 | private static volatile PredictHeavyLoadKeyUtil predictHeavyLoadKeyUtil = null; 20 | 21 | private SynopsisHashMap predictHeavyLoadKeyMap = new SynopsisHashMap(); 22 | 23 | private long dumpKeyCount=0L; 24 | (PredictHeavyLoadKeyUtil.class); 25 | 26 | private long totalDelayTime=0L;// The total actual delay of the winning key 27 | private long totalKeyCount=0L;// Statistics the total number of winning keys 28 | 29 | private Random random=new Random(); 30 | 31 | public PredictHeavyLoadKeyUtil() { 32 | } 33 | 34 | public static PredictHeavyLoadKeyUtil getPredictHeavyLoadKeyUtilInstance(){ 35 | if(null == predictHeavyLoadKeyUtil) 36 | { 37 | synchronized (PredictHeavyLoadKeyUtil.class) 38 | { 39 | predictHeavyLoadKeyUtil=new PredictHeavyLoadKeyUtil(); 40 | } 41 | } 42 | return predictHeavyLoadKeyUtil; 43 | } 44 | 45 | /** 46 | * Throw coins until the coin looks up before the number of coins cast 47 | * @return 48 | */ 49 | public int countCointUtilUp(){ 50 | int rand = (int)(Math.random()*2); 51 | int count = 0; 52 | while(rand == 0 && count < Threshold_l) //Max length set equal to max length+r; 53 | { 54 | rand = (int)(Math.random()*2); 55 | count++; 56 | } 57 | return count; 58 | } 59 | 60 | 61 | public void SynopsisHashMapAllDump(DumpRemoveHandler dumpRemoveHandler) { 62 | int dumpsize = (int) (1 / Threshold_p); 63 | dumpKeyCount++; 64 | if (dumpKeyCount == dumpsize) { 65 | //dump all key 66 | Iterator> iterator = predictHeavyLoadKeyMap.newEntryIterator(); 67 | while (iterator.hasNext()){ 68 | Map.Entry next = iterator.next(); 69 | BitSet bitm = next.getValue(); 70 | String key = next.getKey(); 71 | if(key!=null){ 72 | long[] lo = bitm.toLongArray(); 73 | if(lo.length > 0){ 74 | for(int j=0;j>> 1; 76 | lo[j] = lo[j] | (lo[j+1] << 63); 77 | } 78 | lo[lo.length-1] = lo[lo.length-1] >>> 1; 79 | } 80 | bitm = BitSet.valueOf(lo); 81 | if (bitm.isEmpty()) { 82 | iterator.remove(); 83 | dumpRemoveHandler.dumpRemove(key); 84 | }else 85 | next.setValue(bitm); 86 | } 87 | } 88 | dumpKeyCount = 0; 89 | } 90 | } 91 | 92 | public void SynopsisHashMapRandomDump(DumpRemoveHandler dumpRemoveHandler) { 93 | int size=predictHeavyLoadKeyMap.size; 94 | long startTimeSystemTime=System.currentTimeMillis(); 95 | Iterator> iterator = predictHeavyLoadKeyMap.newEntryIterator(); 96 | while (iterator.hasNext()){ 97 | Map.Entry next = iterator.next(); 98 | if (random.nextDouble()> Threshold_p){ 99 | continue; 100 | } 101 | BitSet bitm = next.getValue(); 102 | String key = next.getKey(); 103 | if(key!=null){ 104 | long[] lo = bitm.toLongArray(); 105 | if(lo.length > 0){ 106 | for(int j=0;j>> 1; 108 | lo[j] = lo[j] | (lo[j+1] << 63); 109 | } 110 | lo[lo.length-1] = lo[lo.length-1] >>> 1; 111 | } 112 | bitm = BitSet.valueOf(lo); 113 | if (bitm.isEmpty()) { 114 | iterator.remove(); 115 | dumpRemoveHandler.dumpRemove(key); 116 | }else 117 | next.setValue(bitm); 118 | } 119 | } 120 | } 121 | 122 | /** 123 | * PredictHeavyLoadKey 124 | * @param key 125 | * @param coninCount 126 | */ 127 | public void PredictHeavyLoadKey(String key,int coninCount){ 128 | int count=coninCount-Threshold_r; 129 | BitSet bitmap=null; 130 | if(predictHeavyLoadKeyMap.get(key)!=null) 131 | bitmap = (BitSet) predictHeavyLoadKeyMap.get(key); 132 | else 133 | bitmap=new BitSet(Threshold_l); 134 | 135 | bitmap.set(coninCount); 136 | predictHeavyLoadKeyMap.put(key,bitmap); 137 | 138 | if(bitmap.cardinality() >= 2) 139 | { 140 | } 141 | } 142 | 143 | public void simpleComputPredictHeavyLoadKey(String key) { 144 | int count = countCointUtilUp(); 145 | int dumpsize = (int) (1 / Threshold_p); 146 | 147 | if (count >= Threshold_r) { 148 | PredictHeavyLoadKey(key, count - Threshold_r); 149 | SynopsisHashMapAllDump(new DumpRemoveHandler() { 150 | @Override 151 | public void dumpRemove(String key) { 152 | 153 | } 154 | }); 155 | } 156 | } 157 | 158 | 159 | public boolean isHeavyLoadKey(String key){ 160 | if(!predictHeavyLoadKeyMap.containsKey(key)) 161 | return false; 162 | if(predictHeavyLoadKeyMap.get(key).cardinality() >= 2) 163 | return true; 164 | else 165 | return false; 166 | } 167 | 168 | public long getTotalDelayTime() { 169 | return totalDelayTime; 170 | } 171 | 172 | public void setTotalDelayTime(long totalDelayTime) { 173 | this.totalDelayTime = totalDelayTime; 174 | } 175 | 176 | public long getTotalKeyCount() { 177 | return totalKeyCount; 178 | } 179 | 180 | public void setTotalKeyCount(long totalKeyCount) { 181 | this.totalKeyCount = totalKeyCount; 182 | } 183 | 184 | public SynopsisHashMap getPredictHeavyLoadKeyMap() { 185 | return predictHeavyLoadKeyMap; 186 | } 187 | 188 | public void setPredictHeavyLoadKeyMap(SynopsisHashMap predictHeavyLoadKeyMap) { 189 | this.predictHeavyLoadKeyMap = predictHeavyLoadKeyMap; 190 | } 191 | } 192 | -------------------------------------------------------------------------------- /src/main/java/com/basic/core/util/PropertiesUtil.java: -------------------------------------------------------------------------------- 1 | package com.basic.core.util; 2 | 3 | import java.io.InputStream; 4 | import java.util.Properties; 5 | 6 | /** 7 | * Created by windy on 2019/1/2. 8 | */ 9 | public class PropertiesUtil { 10 | public static final String fileName="/simois.properties"; 11 | 12 | public static Properties pro; 13 | 14 | static{ 15 | pro=new Properties(); 16 | try { 17 | InputStream in = Object. class .getResourceAsStream( fileName ); 18 | pro.load(in); 19 | in.close(); 20 | } catch (Exception e) { 21 | e.printStackTrace(); 22 | } 23 | 24 | } 25 | 26 | public static String getProperties(String name){ 27 | return pro.getProperty(name); 28 | } 29 | 30 | public static void setProperties(String name ,String value){ 31 | pro.setProperty(name, value); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /src/main/java/com/basic/core/util/SchemaUtils.java: -------------------------------------------------------------------------------- 1 | package com.basic.core.util; 2 | 3 | import java.util.List; 4 | 5 | import com.google.common.base.Splitter; 6 | import static com.google.common.base.Preconditions.checkState; 7 | import com.google.common.collect.ImmutableList; 8 | 9 | public class SchemaUtils 10 | { 11 | public static List parseSchema(String schemaDesp) { 12 | final Splitter splitter = Splitter.on(',').trimResults(); 13 | 14 | return ImmutableList.copyOf(splitter.split(schemaDesp)); 15 | } 16 | 17 | public static int getFieldIdx(String schemaDesp, String field) { 18 | List schema = parseSchema(schemaDesp); 19 | 20 | int idx = schema.indexOf(field.trim()); 21 | 22 | checkState(idx >= 0, "No \"" + field + "\" in " + schema); 23 | return idx; 24 | } 25 | } -------------------------------------------------------------------------------- /src/main/java/com/basic/core/util/Stopwatch.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (C) 2008 The Guava Authors 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | * 16 | * NOTE: This code is branched from the Google Guava v16.0.1 17 | */ 18 | 19 | package com.basic.core.util; 20 | 21 | import java.util.concurrent.TimeUnit; 22 | import static java.util.concurrent.TimeUnit.DAYS; 23 | import static java.util.concurrent.TimeUnit.HOURS; 24 | import static java.util.concurrent.TimeUnit.MICROSECONDS; 25 | import static java.util.concurrent.TimeUnit.MILLISECONDS; 26 | import static java.util.concurrent.TimeUnit.MINUTES; 27 | import static java.util.concurrent.TimeUnit.NANOSECONDS; 28 | import static java.util.concurrent.TimeUnit.SECONDS; 29 | 30 | import com.google.common.base.Ticker; 31 | import static com.google.common.base.Preconditions.checkNotNull; 32 | import static com.google.common.base.Preconditions.checkState; 33 | 34 | public final class Stopwatch 35 | { 36 | private final Ticker ticker; 37 | private boolean isRunning; 38 | private long elapsedNanos; 39 | private long startTick; 40 | 41 | public static Stopwatch createUnstarted() { 42 | return new Stopwatch(); 43 | } 44 | 45 | public static Stopwatch createUnstarted(Ticker ticker) { 46 | return new Stopwatch(ticker); 47 | } 48 | 49 | public static Stopwatch createStarted() { 50 | return new Stopwatch().start(); 51 | } 52 | 53 | public static Stopwatch createStarted(Ticker ticker) { 54 | return new Stopwatch(ticker).start(); 55 | } 56 | 57 | private Stopwatch() { 58 | this(Ticker.systemTicker()); 59 | } 60 | 61 | private Stopwatch(Ticker ticker) { 62 | this.ticker = checkNotNull(ticker, "ticker"); 63 | } 64 | 65 | public boolean isRunning() { 66 | return isRunning; 67 | } 68 | 69 | public Stopwatch start() { 70 | checkState(!isRunning, "This stopwatch is already running."); 71 | isRunning = true; 72 | startTick = ticker.read(); 73 | return this; 74 | } 75 | 76 | public Stopwatch stop() { 77 | long tick = ticker.read(); 78 | checkState(isRunning, "This stopwatch is already stopped."); 79 | isRunning = false; 80 | elapsedNanos += tick - startTick; 81 | return this; 82 | } 83 | 84 | public Stopwatch reset() { 85 | elapsedNanos = 0; 86 | isRunning = false; 87 | return this; 88 | } 89 | 90 | private long elapsedNanos() { 91 | return isRunning ? ticker.read() - startTick + elapsedNanos 92 | : elapsedNanos; 93 | } 94 | 95 | public long elapsed(TimeUnit desiredUnit) { 96 | return desiredUnit.convert(elapsedNanos(), NANOSECONDS); 97 | } 98 | 99 | @Override 100 | public String toString() { 101 | long nanos = elapsedNanos(); 102 | 103 | TimeUnit unit = chooseUnit(nanos); 104 | double value = (double) nanos / NANOSECONDS.convert(1, unit); 105 | 106 | // Too bad this functionality is not exposed as a regular method call 107 | return String.format("%.4g %s", value, abbreviate(unit)); 108 | } 109 | 110 | private static TimeUnit chooseUnit(long nanos) { 111 | if (DAYS.convert(nanos, NANOSECONDS) > 0) { 112 | return DAYS; 113 | } 114 | if (HOURS.convert(nanos, NANOSECONDS) > 0) { 115 | return HOURS; 116 | } 117 | if (MINUTES.convert(nanos, NANOSECONDS) > 0) { 118 | return MINUTES; 119 | } 120 | if (SECONDS.convert(nanos, NANOSECONDS) > 0) { 121 | return SECONDS; 122 | } 123 | if (MILLISECONDS.convert(nanos, NANOSECONDS) > 0) { 124 | return MILLISECONDS; 125 | } 126 | if (MICROSECONDS.convert(nanos, NANOSECONDS) > 0) { 127 | return MICROSECONDS; 128 | } 129 | return NANOSECONDS; 130 | } 131 | 132 | private static String abbreviate(TimeUnit unit) { 133 | switch (unit) { 134 | case NANOSECONDS: 135 | return "ns"; 136 | case MICROSECONDS: 137 | return "\u03bcs"; 138 | case MILLISECONDS: 139 | return "ms"; 140 | case SECONDS: 141 | return "s"; 142 | case MINUTES: 143 | return "min"; 144 | case HOURS: 145 | return "h"; 146 | case DAYS: 147 | return "d"; 148 | default: 149 | throw new AssertionError(); 150 | } 151 | } 152 | } -------------------------------------------------------------------------------- /src/main/java/com/basic/core/util/StormRunner.java: -------------------------------------------------------------------------------- 1 | package com.basic.core.util; 2 | 3 | //import backtype.storm.Config; 4 | //import backtype.storm.LocalCluster; 5 | //import static backtype.storm.StormSubmitter.submitTopology; 6 | import static org.apache.storm.StormSubmitter.submitTopology; 7 | 8 | //import backtype.storm.generated.StormTopology; 9 | import org.apache.storm.Config; 10 | import org.apache.storm.LocalCluster; 11 | import org.apache.storm.generated.StormTopology; 12 | 13 | public final class StormRunner 14 | { 15 | private static final int MILLIS_IN_SEC = 1000; 16 | 17 | public static void runLocally(String topologyName, StormTopology topology, 18 | Config conf, int runtimeInSeconds) throws Exception { 19 | LocalCluster cluster = new LocalCluster(); 20 | cluster.submitTopology(topologyName, conf, topology); 21 | Thread.sleep((long) runtimeInSeconds * MILLIS_IN_SEC); 22 | cluster.killTopology(topologyName); 23 | cluster.shutdown(); 24 | } 25 | 26 | public static void runInCluster(String topologyName, 27 | StormTopology topology, Config conf) throws Exception { 28 | submitTopology(topologyName, conf, topology); 29 | } 30 | } -------------------------------------------------------------------------------- /src/main/java/com/basic/core/util/SynopsisHashMap.java: -------------------------------------------------------------------------------- 1 | package com.basic.core.util; 2 | import java.io.IOException; 3 | import java.io.ObjectInputStream; 4 | import java.io.ObjectOutputStream; 5 | import java.io.Serializable; 6 | import java.util.*; 7 | 8 | /** 9 | * Created by windy on 2019/1/2. 10 | */ 11 | 12 | public class SynopsisHashMap 13 | extends AbstractMap 14 | implements Map, Cloneable, Serializable 15 | { 16 | static final int DEFAULT_INITIAL_CAPACITY = 16; 17 | static final int MAXIMUM_CAPACITY = 1 << 30; 18 | static final float DEFAULT_LOAD_FACTOR = 0.75f; 19 | transient Entry[] table; 20 | transient int size; 21 | int threshold; 22 | final float loadFactor; 23 | transient volatile int modCount; 24 | public SynopsisHashMap(int initialCapacity, float loadFactor) { 25 | if (initialCapacity < 0) 26 | throw new IllegalArgumentException("Illegal initial capacity: " + 27 | initialCapacity); 28 | if (initialCapacity > MAXIMUM_CAPACITY) 29 | initialCapacity = MAXIMUM_CAPACITY; 30 | if (loadFactor <= 0 || Float.isNaN(loadFactor)) 31 | throw new IllegalArgumentException("Illegal load factor: " + 32 | loadFactor); 33 | // Find a power of 2 >= initialCapacity 34 | int capacity = 1; 35 | while (capacity < initialCapacity) 36 | capacity <<= 1; 37 | 38 | this.loadFactor = loadFactor; 39 | threshold = (int)(capacity * loadFactor); 40 | table = new Entry[capacity]; 41 | init(); 42 | } 43 | 44 | public SynopsisHashMap(int initialCapacity) { 45 | this(initialCapacity, DEFAULT_LOAD_FACTOR); 46 | } 47 | 48 | public SynopsisHashMap() { 49 | this.loadFactor = DEFAULT_LOAD_FACTOR; 50 | threshold = (int)(DEFAULT_INITIAL_CAPACITY * DEFAULT_LOAD_FACTOR); 51 | table = new Entry[DEFAULT_INITIAL_CAPACITY]; 52 | init(); 53 | } 54 | 55 | public SynopsisHashMap(Map m) { 56 | this(Math.max((int) (m.size() / DEFAULT_LOAD_FACTOR) + 1, 57 | DEFAULT_INITIAL_CAPACITY), DEFAULT_LOAD_FACTOR); 58 | putAllForCreate(m); 59 | } 60 | 61 | void init() { 62 | } 63 | 64 | static int hash(int h) { 65 | h ^= (h >>> 20) ^ (h >>> 12); 66 | return h ^ (h >>> 7) ^ (h >>> 4); 67 | } 68 | 69 | static int indexFor(int h, int length) { 70 | return h & (length-1); 71 | } 72 | 73 | public int size() { 74 | return size; 75 | } 76 | 77 | public boolean isEmpty() { 78 | return size == 0; 79 | } 80 | 81 | public V get(Object key) { 82 | if (key == null) 83 | return getForNullKey(); 84 | int hash = hash(key.hashCode()); 85 | for (Entry e = table[indexFor(hash, table.length)]; 86 | e != null; 87 | e = e.next) { 88 | Object k; 89 | if (e.hash == hash && ((k = e.key) == key || key.equals(k))) 90 | return e.value; 91 | } 92 | return null; 93 | } 94 | 95 | public K getrandomkey() { 96 | int i; 97 | Entry e = null; 98 | Random rand =new Random(); 99 | while (e==null) { 100 | i = rand.nextInt(table.length); 101 | e = table[i]; 102 | } 103 | return e.key; 104 | } 105 | 106 | private V getForNullKey() { 107 | for (Entry e = table[0]; e != null; e = e.next) { 108 | if (e.key == null) 109 | return e.value; 110 | } 111 | return null; 112 | } 113 | 114 | public boolean containsKey(Object key) { 115 | return getEntry(key) != null; 116 | } 117 | 118 | final Entry getEntry(Object key) { 119 | int hash = (key == null) ? 0 : hash(key.hashCode()); 120 | for (Entry e = table[indexFor(hash, table.length)]; 121 | e != null; 122 | e = e.next) { 123 | Object k; 124 | if (e.hash == hash && 125 | ((k = e.key) == key || (key != null && key.equals(k)))) 126 | return e; 127 | } 128 | return null; 129 | } 130 | 131 | public V put(K key, V value) { 132 | if (key == null) 133 | return putForNullKey(value); 134 | int hash = hash(key.hashCode()); 135 | int i = indexFor(hash, table.length); 136 | for (Entry e = table[i]; e != null; e = e.next) { 137 | Object k; 138 | if (e.hash == hash && ((k = e.key) == key || key.equals(k))) { 139 | V oldValue = e.value; 140 | e.value = value; 141 | e.recordAccess(this); 142 | return oldValue; 143 | } 144 | } 145 | modCount++; 146 | addEntry(hash, key, value, i); 147 | return null; 148 | } 149 | 150 | private V putForNullKey(V value) { 151 | for (Entry e = table[0]; e != null; e = e.next) { 152 | if (e.key == null) { 153 | V oldValue = e.value; 154 | e.value = value; 155 | e.recordAccess(this); 156 | return oldValue; 157 | } 158 | } 159 | modCount++; 160 | addEntry(0, null, value, 0); 161 | return null; 162 | } 163 | 164 | private void putForCreate(K key, V value) { 165 | int hash = (key == null) ? 0 : hash(key.hashCode()); 166 | int i = indexFor(hash, table.length); 167 | for (Entry e = table[i]; e != null; e = e.next) { 168 | Object k; 169 | if (e.hash == hash && 170 | ((k = e.key) == key || (key != null && key.equals(k)))) { 171 | e.value = value; 172 | return; 173 | } 174 | } 175 | createEntry(hash, key, value, i); 176 | } 177 | 178 | private void putAllForCreate(Map m) { 179 | for (Iterator> i = m.entrySet().iterator(); i.hasNext(); ) { 180 | Map.Entry e = i.next(); 181 | putForCreate(e.getKey(), e.getValue()); 182 | } 183 | } 184 | 185 | void resize(int newCapacity) { 186 | Entry[] oldTable = table; 187 | int oldCapacity = oldTable.length; 188 | if (oldCapacity == MAXIMUM_CAPACITY) { 189 | threshold = Integer.MAX_VALUE; 190 | return; 191 | } 192 | 193 | Entry[] newTable = new Entry[newCapacity]; 194 | transfer(newTable); 195 | table = newTable; 196 | threshold = (int)(newCapacity * loadFactor); 197 | } 198 | 199 | void transfer(Entry[] newTable) { 200 | Entry[] src = table; 201 | int newCapacity = newTable.length; 202 | for (int j = 0; j < src.length; j++) { 203 | Entry e = src[j]; 204 | if (e != null) { 205 | src[j] = null; 206 | do { 207 | Entry next = e.next; 208 | int i = indexFor(e.hash, newCapacity); 209 | e.next = newTable[i]; 210 | newTable[i] = e; 211 | e = next; 212 | } while (e != null); 213 | } 214 | } 215 | } 216 | 217 | public void putAll(Map m) { 218 | int numKeysToBeAdded = m.size(); 219 | if (numKeysToBeAdded == 0) 220 | return; 221 | if (numKeysToBeAdded > threshold) { 222 | int targetCapacity = (int)(numKeysToBeAdded / loadFactor + 1); 223 | if (targetCapacity > MAXIMUM_CAPACITY) 224 | targetCapacity = MAXIMUM_CAPACITY; 225 | int newCapacity = table.length; 226 | while (newCapacity < targetCapacity) 227 | newCapacity <<= 1; 228 | if (newCapacity > table.length) 229 | resize(newCapacity); 230 | } 231 | for (Iterator> i = m.entrySet().iterator(); i.hasNext(); ) { 232 | Map.Entry e = i.next(); 233 | put(e.getKey(), e.getValue()); 234 | } 235 | } 236 | 237 | public V remove(Object key) { 238 | Entry e = removeEntryForKey(key); 239 | return (e == null ? null : e.value); 240 | } 241 | 242 | final Entry removeEntryForKey(Object key) { 243 | int hash = (key == null) ? 0 : hash(key.hashCode()); 244 | int i = indexFor(hash, table.length); 245 | Entry prev = table[i]; 246 | Entry e = prev; 247 | 248 | while (e != null) { 249 | Entry next = e.next; 250 | Object k; 251 | if (e.hash == hash && 252 | ((k = e.key) == key || (key != null && key.equals(k)))) { 253 | modCount++; 254 | size--; 255 | if (prev == e) 256 | table[i] = next; 257 | else 258 | prev.next = next; 259 | e.recordRemoval(this); 260 | return e; 261 | } 262 | prev = e; 263 | e = next; 264 | } 265 | 266 | return e; 267 | } 268 | 269 | final Entry removeMapping(Object o) { 270 | if (!(o instanceof Map.Entry)) 271 | return null; 272 | 273 | Map.Entry entry = (Map.Entry) o; 274 | Object key = entry.getKey(); 275 | int hash = (key == null) ? 0 : hash(key.hashCode()); 276 | int i = indexFor(hash, table.length); 277 | Entry prev = table[i]; 278 | Entry e = prev; 279 | 280 | while (e != null) { 281 | Entry next = e.next; 282 | if (e.hash == hash && e.equals(entry)) { 283 | modCount++; 284 | size--; 285 | if (prev == e) 286 | table[i] = next; 287 | else 288 | prev.next = next; 289 | e.recordRemoval(this); 290 | return e; 291 | } 292 | prev = e; 293 | e = next; 294 | } 295 | 296 | return e; 297 | } 298 | 299 | public void clear() { 300 | modCount++; 301 | Entry[] tab = table; 302 | for (int i = 0; i < tab.length; i++) 303 | tab[i] = null; 304 | size = 0; 305 | } 306 | 307 | public boolean containsValue(Object value) { 308 | if (value == null) 309 | return containsNullValue(); 310 | 311 | Entry[] tab = table; 312 | for (int i = 0; i < tab.length ; i++) 313 | for (Entry e = tab[i] ; e != null ; e = e.next) 314 | if (value.equals(e.value)) 315 | return true; 316 | return false; 317 | } 318 | 319 | private boolean containsNullValue() { 320 | Entry[] tab = table; 321 | for (int i = 0; i < tab.length ; i++) 322 | for (Entry e = tab[i] ; e != null ; e = e.next) 323 | if (e.value == null) 324 | return true; 325 | return false; 326 | } 327 | 328 | public Object clone() { 329 | SynopsisHashMap result = null; 330 | try { 331 | result = (SynopsisHashMap)super.clone(); 332 | } catch (CloneNotSupportedException e) { 333 | // assert false; 334 | } 335 | result.table = new Entry[table.length]; 336 | result.entrySet = null; 337 | result.modCount = 0; 338 | result.size = 0; 339 | result.init(); 340 | result.putAllForCreate(this); 341 | 342 | return result; 343 | } 344 | 345 | static class Entry implements Map.Entry { 346 | final K key; 347 | V value; 348 | Entry next; 349 | final int hash; 350 | Entry(int h, K k, V v, Entry n) { 351 | value = v; 352 | next = n; 353 | key = k; 354 | hash = h; 355 | } 356 | public final K getKey() { 357 | return key; 358 | } 359 | public final V getValue() { 360 | return value; 361 | } 362 | public final V setValue(V newValue) { 363 | V oldValue = value; 364 | value = newValue; 365 | return oldValue; 366 | } 367 | public final boolean equals(Object o) { 368 | if (!(o instanceof Map.Entry)) 369 | return false; 370 | Map.Entry e = (Map.Entry)o; 371 | Object k1 = getKey(); 372 | Object k2 = e.getKey(); 373 | if (k1 == k2 || (k1 != null && k1.equals(k2))) { 374 | Object v1 = getValue(); 375 | Object v2 = e.getValue(); 376 | if (v1 == v2 || (v1 != null && v1.equals(v2))) 377 | return true; 378 | } 379 | return false; 380 | } 381 | public final int hashCode() { 382 | return (key==null ? 0 : key.hashCode()) ^ 383 | (value==null ? 0 : value.hashCode()); 384 | } 385 | public final String toString() { 386 | return getKey() + "=" + getValue(); 387 | } 388 | void recordAccess(SynopsisHashMap m) { 389 | } 390 | void recordRemoval(SynopsisHashMap m) { 391 | } 392 | } 393 | 394 | void addEntry(int hash, K key, V value, int bucketIndex) { 395 | Entry e = table[bucketIndex]; 396 | table[bucketIndex] = new Entry(hash, key, value, e); 397 | // size++; 398 | if (size++ >= threshold) 399 | resize(2 * table.length); 400 | } 401 | 402 | void createEntry(int hash, K key, V value, int bucketIndex) { 403 | Entry e = table[bucketIndex]; 404 | table[bucketIndex] = new Entry(hash, key, value, e); 405 | size++; 406 | } 407 | 408 | public abstract class HashIterator implements Iterator { 409 | Entry next; // next entry to return 410 | int expectedModCount; // For fast-fail 411 | int index; // current slot 412 | Entry current; // current entry 413 | HashIterator() { 414 | expectedModCount = modCount; 415 | if (size > 0) { // advance to first entry 416 | Entry[] t = table; 417 | while (index < t.length && (next = t[index++]) == null) 418 | ; 419 | } 420 | } 421 | public final boolean hasNext() { 422 | return next != null; 423 | } 424 | final Entry nextEntry() { 425 | if (modCount != expectedModCount) 426 | throw new ConcurrentModificationException(); 427 | Entry e = next; 428 | if (e == null) 429 | throw new NoSuchElementException(); 430 | 431 | if ((next = e.next) == null) { 432 | Entry[] t = table; 433 | while (index < t.length && (next = t[index++]) == null) 434 | ; 435 | } 436 | current = e; 437 | return e; 438 | } 439 | public void remove() { 440 | if (current == null) 441 | throw new IllegalStateException(); 442 | if (modCount != expectedModCount) 443 | throw new ConcurrentModificationException(); 444 | Object k = current.key; 445 | current = null; 446 | SynopsisHashMap.this.removeEntryForKey(k); 447 | expectedModCount = modCount; 448 | } 449 | } 450 | 451 | private final class ValueIterator extends HashIterator { 452 | public V next() { 453 | return nextEntry().value; 454 | } 455 | } 456 | 457 | private final class KeyIterator extends HashIterator { 458 | public K next() { 459 | return nextEntry().getKey(); 460 | } 461 | } 462 | 463 | private final class EntryIterator extends HashIterator> { 464 | public Map.Entry next() { 465 | return nextEntry(); 466 | } 467 | } 468 | 469 | // Subclass overrides these to alter behavior of views' iterator() method 470 | Iterator newKeyIterator() { 471 | return new KeyIterator(); 472 | } 473 | Iterator newValueIterator() { 474 | return new ValueIterator(); 475 | } 476 | Iterator> newEntryIterator() { 477 | return new EntryIterator(); 478 | } 479 | 480 | private transient Set> entrySet = null; 481 | 482 | public Set> entrySet() { 483 | return entrySet0(); 484 | } 485 | 486 | private Set> entrySet0() { 487 | Set> es = entrySet; 488 | return es != null ? es : (entrySet = new EntrySet()); 489 | } 490 | 491 | private final class EntrySet extends AbstractSet> { 492 | public Iterator> iterator() { 493 | return newEntryIterator(); 494 | } 495 | public boolean contains(Object o) { 496 | if (!(o instanceof Map.Entry)) 497 | return false; 498 | Map.Entry e = (Map.Entry) o; 499 | Entry candidate = getEntry(e.getKey()); 500 | return candidate != null && candidate.equals(e); 501 | } 502 | public boolean remove(Object o) { 503 | return removeMapping(o) != null; 504 | } 505 | public int size() { 506 | return size; 507 | } 508 | public void clear() { 509 | SynopsisHashMap.this.clear(); 510 | } 511 | } 512 | 513 | private void writeObject(ObjectOutputStream s) 514 | throws IOException 515 | { 516 | Iterator> i = 517 | (size > 0) ? entrySet0().iterator() : null; 518 | 519 | // Write out the threshold, loadfactor, and any hidden stuff 520 | s.defaultWriteObject(); 521 | 522 | // Write out number of buckets 523 | s.writeInt(table.length); 524 | 525 | // Write out size (number of Mappings) 526 | s.writeInt(size); 527 | 528 | // Write out keys and values (alternating) 529 | if (i != null) { 530 | while (i.hasNext()) { 531 | Map.Entry e = i.next(); 532 | s.writeObject(e.getKey()); 533 | s.writeObject(e.getValue()); 534 | } 535 | } 536 | } 537 | 538 | private static final long serialVersionUID = 362498820763181265L; 539 | 540 | private void readObject(ObjectInputStream s) 541 | throws IOException, ClassNotFoundException 542 | { 543 | // Read in the threshold, loadfactor, and any hidden stuff 544 | s.defaultReadObject(); 545 | 546 | // Read in number of buckets and allocate the bucket array; 547 | int numBuckets = s.readInt(); 548 | table = new Entry[numBuckets]; 549 | 550 | init(); // Give subclass a chance to do its thing. 551 | 552 | // Read in size (number of Mappings) 553 | int size = s.readInt(); 554 | 555 | // Read the keys and values, and put the mappings in the HashMap 556 | for (int i=0; i