JStorm English introduction JStorm Chinese introduction
- Support exactly-once with async checkpoint via rocksdb and HDFS.
- Introduce new window mechanism
- supports tumbling window and sliding window.
- supports count window, processing time window, event time window, session window.
- doesn't hold all data before a window is triggered, computes on data arrival.
- Support gray upgrade
- supports per worker/component gray upgrade
- supports upgrade rollback
- Add memory/rocksdb-based KV store.
- HBase metrics plugin is open source
- Support multiple metrics uploaders.
- Add api in MetricClient to register topology-level metrics
- Support component stream metrics, i.e., stream metrics aggregated in components
- Support deserialize for no-arg class in kryo
- Add getValue method in AsmMetric for quick assert so that unit tests/integration tests don't have to get metrics from nimbus
- Fix the bug of incorrect computation of unstopped tasks when assigning topology
- Fix the bug that supervisor storm.yaml is always different from nimbus storm.yaml
- Fix the bug that kryo doesn't accept conf value of literal string "true"/"false"
- Thanks to @gohitbear @bryant1410 @oubenruing for doc fixes.
- Thanks to @zeromem @elloray @yunfan123 @iBuddha @Glowdable @waooog for bug fixes.
- Performance is improved by 200%~300%, compared to Release 2.1.1 and 0.9.8.1 in several testing scenarios, while
120%~200% compared to Flink and 300%~400% compared to Storm.
- Restructure the batch solution
- Improve serialization and deserialization to reduce the cost of cpu and network
- Improve the cost of cpu on critical path and metrics
- Improve the strategy of netty client and netty server
- Support consume and publish of disruptor queue under batch mode
- Introduce snapshot exactly once framework
- Compared to Trident solution, the performance of new framework is increased by several times. Besides it.
- The new framework also support "at least once" mode. Compared to the acker mechanism,it will reduce the cost of relative calculation in acker, and the cost of network, which will improve the performance singificantly.
- Support JStorm on yarn。
- Currently, jstorm cluster is capable of fast deployments,and fast scale-in/scale-out. It will improve the utility of resource.
- Re-design the solution of backpressure. Currently, the flow control is stage by stage。
- The solution is simple and effective now. The response is much more faster when the exchange of switch on/off of backpressure.
- The performance and stability is improved significantly, compared to the original solution.
- Introduce Window API
- Support tumbling window,sliding window
- window support two collection mode, count and duration.
- Support watermark mechanism
- Introduce the support of Flux
- Flux is a programing framework or component which is aim to help create and deploy the topology of jstorm quickly.
- Isolate the dependencies of jstorm and user topology by maven shade plugin to fix the conflict problem.
- Improve Shuffle grouping solution
- Integrate shuffle, localOrShuffle and localFirst. The grouping solution will be auto adapted according to the assignment of topology.
- Introduce load aware in shuffle to ensure the load balance of downstreams.
- Support to configure blacklist in Nimbus to exclude some problematic nodes
- Support batch mode in trident
- Support the push of configuration of cluster
- Add buildTs to supervisor info and heartbeats
- Add ext module for nimbus and supervisor to support external plugins
- Add jstorm-elasticsearch support, thanks to @elloray for your contribution
- Restructure nimbus metrics implementation. Currently, the topology metrics runnable is event-drive.
- Restructure topology master. Currently, the processor in TM is event-drive.
- Add some examples to cover more scenarios
- Disable stream metrics to reduce the cost of sending metrics to Nimbus
- Support metrics in local mode
- Improve the implementation of gauge by changing the instantaneous value of each minute,to the average value of some sample values in each minute.
- Introduce an approximate histogram calculation to reduce memory usage of histogram metrics
- Add Full GC and supervisor network related metrics
- Fix message disorder bug
- Fix the bug that some connections to zookeeper are not closed by expected when encountering exception in supervisor.
- The deactivate might be called by mistake when task init
- The rootId might be duplicated occasionally. It will cause the unexpected message failure.
- Fix the bug when local mode
- Fix logwriter's bug
- Some task metrics(RecvTps ProcessLatency) might not be aggregated correctly.
- Fix the racing condition of AsmCounter during flushing
- 1.5~6X performance boost from worst to best scenarios compared to JStorm-2.1.0
- Add application-level auto-batch
- Add independent control channel to separate control msgs from biz msgs to guarantee high priority for control msgs
- Dramatic performance boost in metrics, see "Improvements" section
- Support jdk1.8
- Add Nimbus hook and topology hook
- Metrics system:
- Support disable/enable metrics on the fly
- Add jstorm metrics design docs, see JSTORM-METRICS.md
- JStorm web UI:
- Add zookeeper viewer in web UI, thanks to @dingjun84
- Add log search and deep log search, support both backward search and forward search
- Support log file download
- Support changing log level on the fly
- Change error structure in zk, add errorLevel, errorCode and duration.
- Add supervisor health check
- Add -Dexclude.jars option to enable filtering jars manually
- Metrics:
- use JHistogram/JMeter instead of Histogram/Meter, change internal Clock.tick to System.currentTimeMillis to improve performance (50+% boost in Meter and 25%+ boost in Histogram)
- add TupleLifeCycle metric
- add supervisor metrics: total_cpu_usage, total_mem_usage, disk_usage
- remove some unnecessary metrics like emitTime, etc.
- Use HeapByteBuffer instead of List to transmit metric data points, reduce 60+% metrics memory usage
- Change sample rate from 10% to 5% by default
- Remove AsmTimer and related code
- Log related:
- Use logback by default instead of log4j, exclude slf4j-log4j12 dependency
- Use jstorm.log.dir property instead of ${jstorm.home}/logs, see jstorm.logback.xml
- Change all log4j Logger's to slf4j Logger's
- Set default log page size(log.page.size) in defaults.yaml to 128KB (web UI)
- Change topology log structure, add ${topology.name} directory, see jstorm.logback.xml
- Add timestamp in supervisor/nimbus gc log files; backup worker gc log before launching a new worker;
- Set logback/log4j file encoding to UTF-8
- Refine backpressure stragety to avoid over-backpressure
- Change acker pending rotating map to single thread to improve performance
- Update RefreshConnections to avoid downloading assignments from zk frequently
- Change default memory of Supervisor to 1G (previous 512MB)
- Use ProcessLauncher to launch processes
- Add DefaultUncaughtExceptionHandler for supervisor and nimbus
- Change local ports to be different from 0.9.x versions (supervisor.slots.ports.base, nimbus.thrift.port, nimbus.deamon.logview.port, supervisor.deamon.logview.port)
- Change highcharts to echarts to avoid potential license violation
- Dependency upgrades:
- Upgrade kryo to 2.23.0
- Upgrade disruptor to 3.2.2
- Fix deadlock when starting workers
- Fix the bug that when localstate file is empty, supervisor can't start
- Fix kryo serialization for HeapByteBuffer in metrics
- Fix total memory usage calculation
- Fix the bug that empty worker is assigned when configured worker number is bigger than the actual number for user defined scheduler
- Fix UI log home directory
- Fix XSS security bug in web UI
- Don't start TopologyMetricsRunnable thread in local mode, thanks to @L-Donne
- Fix JSTORM-141, JSTORM-188 that TopologyMetricsRunnable consumes too much CPU
- Remove MaxTenuringThreshold JVM option support jdk1.8, thanks to @249550148
- Fix possible NPE in MkLocalShuffer
- Add cleanup for core dumps
- Add supervisor health check in healthCheck.sh
- Change jstorm.py to terminate the original python process when starting nimbus/supervisor
- JStorm 2.1.1 is mostly compatible with 2.1.0, but it's better to restart your topologies to finish the upgrade.
- If you're using log4j, be cautious that we have switched default logging system to logback, if you still want to use log4j, please add "user.defined.log4j.conf: jstorm.log4j.properties" to your conf/storm.yaml.
- Totally redesign Web UI
- Make the UI more beautiful
- Improve Web UI speed much.
- Add Cluster/Topology Level Summarized Metrics in recent 30 minutes.
- Add DAG in the Web UI, support Uer Interaction to get key information such as emit, tuple lifecycle, tps
- Redesign Metrics/Monitor System
- New metrics core, support sample with more metric, avoid noise, merge metrics automatically for user.
- No metrics will be stored in ZK
- Support metrics HA
- Add more useful metrics, such as tuple lifecycle, netty metrics, disk space etc. accurately get worker memory
- Support external storage plugin to store metrics.
- Implement Smart BackPressure
- Smart Backpressure, the dataflow will be more stable, avoid noise to trigger
- Easy to manual control Backpressure
- Implement TopologyMaster
- Redesign hearbeat mechanism, easily support 6000+ tasks
- Collect all task's metrics, do merge job, release Nimbus pressure.
- Central Control Coordinator, issue control command
- Redesign ZK usage, one set of ZK support more 2000+ hardware nodes.
- No dynamic data in ZK, such as heartbeat, metrics, monitor status.
- Nimbus reduce visiting ZK frequence when serve thrift API.
- Reduce visiting ZK frequence, merge some task level ZK node.
- Reduce visiting ZK frequence, remove useless ZK node, such as empty taskerror node
- Tuning ZK cache
- Optimize ZK reconnect mechanism
- Tuning Executor Batch performance
- Add smart batch size setting
- Remove memory copy
- Directly issue tuple without batch for internal channel
- Set the default Serialize/Deserialize method as Kryo
- Set the default Serialized/Deserialized method as Kryo to improve performance.
- Support dynamic reload binary/configuration
- Tuning LocalShuffle performance, Set 3 level priority, local worker, local node, other node, add dynamic check queue status, connection status.
- Optimize Nimbus HA, only the highest priority nimbuses can be promoted as master
- Supervisor automatically dump worker jstack/jmap, when worker's status is invalid.
- Supervisor can generate more ports according to memory.
- Supervisor can download binary more time.
- Support set logdir in configuration
- Add configuration "nimbus.host.start.supervisor"
- Add supervisor/nimbus/drpc gc log
- Adjust jvm parameter 1. set -Xmn 1/2 of heap memory 2. set PermSize to 1/32 and MaxPermSize 1/16 of heap memory; 3. set -Xms by "worker.memory.min.size"。
- Refine ZK error schema, when worker is dead, UI will report error
- Add function to zktool utility, support remove all topology znodes, support list
- Optimize netty client.
- Dynamic update connected task status by network connection, not by ZK znode.
- Add configuration "topology.enable.metrics".
- Classify all topology log into one directory by topologyName.
- Skip download same binary when assigment has been changed.
- Skip start worker when binary is invalid.
- Use correct configuration map in a lot of worker thread
- In the first step Nimbus will check topologyName or not when submit topology
- Support fieldGrouping for Object[]
- For drpc single instance under one configuration
- In the client topologyNameExists interface,directly use trhift api
- Fix failed to restart due to topology cleanup thread's competition
- Fix the bug that backpressure might be lost when trigger bolt was failed.
- Fixed the bug that DefaultMetricUploader doesn't delete metrics data in rocksdb, causing new metrics data cannot be appended.
- Optimize cleandisk.sh, avoid delete useful worker log
- Redesign Metric/Monitor system, new RollingWindow/Metrics/NettyMetrics, all data will send/recv through thrift
- Redesign Web-UI, the new Web-UI code is clear and clean
- Add NimbusCache Layer, using RocksDB and TimeCacheWindow
- Refactoring all ZK structure and ZK operation
- Refactoring all thrift structure
- Merge jstorm-client/jstorm-client-extension/jstorm-core 3 modules into jstorm-core
- Set the dependency version same as storm
- Sync apache-storm-0.10.0-beta1 all java code
- Switch log system to logback
- Upgrade thrift to apache thrift 0.9.2
- Performance tuning Huge topology more than 600 workers or 2000 tasks
- Require jdk7 or higher
- Batch the tuples whose target task is same, before sending out(task.batch.tuple=true,task.msg.batch.size=4).
- LocalFirst grouping is updated. If all local tasks are busy, the tasks of outside nodes will be chosen as target task instead of waiting on the busy local task.
- Support user to reload the application config when topology is running.
- Support user to define the task heartbeat timeout and task cleanup timeout for topology.
- Update the wait strategy of disruptor queue to no-blocking mode "TimeoutBlockingWaitStrategy"
- Support user to define the timeout of discarding messages that are pending for a long time in netty buffer.
- Update the message processing structure. The virtualPortDispatch and drainer thread are removed to reduce the unnecessary cost of cpu and the transmitting of tuples
- Add jstorm parameter "--include-jars" when submit topology, add these jar to classpath
- Nimbus or Supervisor suicide when the local ip is 127.0.0.0
- Add user-define-scheduler example
- Merge Supervisor's syncSupervisor and syncProcess
- Improve the GC setting.
- Fix the bug that task heartbeat might not be updated timely in some scenarioes.
- Fix the bug that the reconnection operation might be stick for a unexpected period when the connection to remote worker is shutdown and some messages are buffer in netty.
- Reuse thrift client when submit topology
- Avoid repeatedly download binary when failed to start worker.
- Change task's heartbeat timeout to 4 minutes
- Set the netty client thread pool(clientScheduleService) size as 5
- Improve cleandisk.sh, avoid delete current directory and /tmp/hsperfdata_admin
- Add executable attribute for the script under example
- Add parameter to stat.sh, which can be used to start supervisor or not. This is useful under virtual
- Support dynamic scale-out/scale-in of worker, spout, bolt or acker without stopping the service of topology.
- When enable cgroup, Support the upper limit control of cpu core usage. Default setting is 3 cpu cores.
- Update the mechanism of task heartbeats to make heartbeat to track the status of spout/bolt execute thread correctly.
- Support to add jstorm prefix info(clusterName, topologyName, ip:port, componentName, taskId, taskIndex) for worker/task log
- Check the heartbeat of supervisor when topology assignment to ensure no worker will be assigned into a dead supervisor
- Add api to query the task/worker's metric info, e.g. load status of task queue, worker cpu usage, worker mem usage...
- Try to re-download jars when staring worker fails several times to avoid potential corruption of jars
- Add Nimbus ZK cache, accelerate nimbus read zk
- Add thrift api getVersion, it will be used check between the client jstorm version and the server jstorm version.
- Update the metrics' structure to Alimonitor
- Add exclude-jar parameter into jstorm.py, which avoid class conflict when submit topology
- Fix the no response problem of supervisor process when subimtting big amout topologys in a short time
- When submitting two or more topologys at the same time, the later one might be failed.
- TickTuple does not need to be acked. Fix the incorrect count of failure message.
- Fix the potential incorrect assignment when use.old.assignment=true
- Fix failed to remove some zk nodes when kill topology
- Fix failed to restart topology, when nimbus do assignment job.
- Fix NPE when register metrics
- Fix failed to read ZK monitor znode through zktool
- Fix exception when enable classload and local mode
- Fix duplicate log when enable user-defined logback in local mode
- Set Nimbus jvm memory size as 4G
- Set hearbeat from supervisor to nimbus timeout from 60s to 180s
- In order to avoid OOM, set storm.messaging.netty.max.pending as 4
- Set task queue size as 1024, worker's total send/receive queue size as 2048
- Add rpm build spec
- Add deploy files of jstorm for rpm package building
- Enable the cleandisk cronjob every hour, reserve coredump for only one hour.
- Implement tick tuple
- Support logback
- Support to load the user defined configuration file of log4j
- Enable the display of user defined metrics in web UI
- Add "topologyName" parameter for "jstorm list" command
- Support the use of ip and hostname at the same for user defined schedule
- Support junit test for local mode
- Enable client command(e.g. jstorm jar) to load self-defined storm.yaml
- Add activate and deactivate api of spout, which are used in nextTuple prepare phase
- Update the support of multi language
- Check the worker's heartbeat asynchronously to speed up the lunch of worker
- Add the check of worker's pid to speed up the detect of dead worker
- Fix the high cpu load of disruptor producer when disruptor queue is full
- Remove the confused exception reported by disruptor queue when killing worker
- Fix the failure problem of "jstorm restart" client command
- Report error when user submits the jar built on a incompatible jstorm release
- Fix the problem that one log will printed twice when user define a configuration of log4j or logback on local mode
- Fix the potential exception when killing topology on local mode
- Forbid user to change the log level of jstorm log
- Add a configuration template of logback
- Fix the problem that process the upload of lib jar as application jar
- Makesure the clean of ZK node for a topology which is removed
- Add the information of topology name when java core dump
- Fix the incorrect value of -XX:MaxTenuringThreshold. Currently, the default value of jstorm is 20, but the max value in JDK8 is 15.
- Fix the potential reading failure of cpu core number, which may cause the supervisor slot to be set to 0
- Fix the "Address family not supported by protocol family" error on local mode
- Do not start logview http server on local mode
- Add the creation of log dir in supervisor alive checking scription
- Check the correctness of ip specified in configuration file before starting nimbus
- Check the correctness of env variable $JAVA_HOME/$JSTORM_HOME/$JSTORM_CONF_DIR before starting jstorm service
- Specify the log dir for rpm installation
- Add reading permission of /home/admin/jstorm and /home/admin/logs for all users after rpm installation
- Config local temporay ports when rpm installation
- Add noarch rpm package
- Add option to switch between BlockingQueue and Disruptor
- Fix the bug which under sync netty mode, client failed to send message to server
- Fix the bug let web UI can dispaly 0.9.6.1 cluster
- Fix the bug topology can be submited without main jar but a lot of little jar
- Fix the bug restart command
- Fix the bug trident bug
- Add the validation of topology name, component name... Only A-Z, a-z, 0-9, '_', '-', '.' are valid now.
- Fix the bug close thrift client
- Improve user experience from Web UI 1.1 Add jstack link 1.2 Add worker log link in supervisor page 1.3 Add Web UI log encode setting "gbk" or "utf-8" 1.4 Show starting tasks in component page 1.5 Show dead task's information in UI 1.6 Fix the bug that error info can not be displayed in UI when task is restarting
- Add restart command, with this command, user can reload configuration, reset worker/task parallism
- Upgrade curator/disruptor/guava version
- Revert json lib to google-simple json, wrap all json operation into two utility method
- Add new storm submit api, supporting submit topology under java
- Enable launch process with backend method
- Set "spout.pending.full.sleep" default value as true
- Fix the bug user define sceduler not support a list of workers
- Add disruptor/JStormUtils junit test
- Enable user to configure the name of monitor name of alimonitor
- Add tcp option "reuseAddress" in netty framework
- Fix the bug: When spout does not implement the ICommitterTrident interface, MasterCoordinatorSpout will stick on commit phase.
- Improve user experience from Web UI 1.1 Add jstack link 1.2 Add worker log link in supervisor page 1.3 Add Web UI log encode setting "gbk" or "utf-8" 1.4 Show starting tasks in component page 1.5 Show dead task's information in UI 1.6 Fix the bug that error info can not be displayed in UI when task is restarting
- Add restart command, with this command, user can reload configuration, reset worker/task parallism
- Upgrade curator/disruptor/guava version
- Revert json lib to google-simple json, wrap all json operation into two utility method
- Add new storm submit api, supporting submit topology under java
- Enable launch process with backend method
- Set "spout.pending.full.sleep" default value as true
- Fix the bug user define sceduler not support a list of workers
- Add disruptor/JStormUtils junit test
- Enable user to configure the name of monitor name of alimonitor
- Add tcp option "reuseAddress" in netty framework
- Fix the bug: When spout does not implement the ICommitterTrident interface, MasterCoordinatorSpout will stick on commit phase.
- Add management of multiclusters to Web UI. Added management tools for multiclusters in WebUI.
- Merged Trident API from storm-0.9.3
- Replaced gson with fastjson
- Refactored metric json generation code.
- Stored version info with $JSTORM_HOME/RELEASE.
- Replaced SingleThreadDisruptorQueue with MultiThreadDisruptorQueue in task deserialize thread.
- Fixed issues with worker count on Web UI.
- Fixed issues with accessing the task map with multi-threads.
- Fixed NullPointerException while killing worker and reading worker's hearbeat object.
- Netty client connect to server only in NettyClient module.
- Add break loop operation when netty client connection is closed
- Fix the bug that topology warning flag present in cluster page is not consistent with error information present in topology page
- Add recovery function when the data of task error information is corrupted
- Fix the bug that the metric data can not be uploaded onto Alimonitor when ugrading from pre-0.9.6 to 0.9.6 and executing pkill java without restart the topologying
- Fix the bug that zeroMq failed to receive data
- Add interface to easily setting worker's memory
- Set default value of topology.alimonitor.metrics.post to false
- Only start NETTY_SERVER_DECODE_TIME for netty server
- Keep compatible with Storm for local mode
- Print rootId when tuple failed
- In order to keep compatible with Storm, add submitTopologyWithProgressBar interface
- Upgrade netty version from 3.2.7 to 3.9.0
- Support assign topology to user-defined supervisors
- Update UI
- Display the metrics information of task and worker
- Add warning flag when errors occur for a topology
- Add link from supervisor page to task page
- Send metrics data to Alimonitor
- Add metrics interface for user
- Add task.cleanup.timeout.sec setting to let task gently cleanup
- Set the worker's log name as topologyName-worker-port.log
- Add setting "worker.redirect.output.file", so worker can redirect System.out/System.err to one setting file
- Add storm list command
- Add closing channel check in netty client to avoid double close
- Add connecting check in netty client to avoid connecting one server twice at one time
- Add netty sync mode
- Add block operation in netty async mode
- Replace exception with Throwable in executor layer
- Upgrade curator-framework version from 1.15 to 1.3.2
- Add more netty junit test
- Add log when queue is full
- Redesign scheduler arithmetic, basing worker not task .
- Fix disruptor use too much cpu
- Add target NettyServer log when f1ail to send data by netty
- Improve speed between tasks who is running in one worker
- Fix wrong timeout seconds
- Add checking port when worker initialize and begin to kill old worker
- Move worker hearbeat thread before initializing tasks
- Move init netty-server before initializeing tasks
- Check whether tuple's rootId is duplicated
- Add default value into Utils.getInt
- Add result function in ReconnectRunnable
- Add operation to start Timetick
- Halt process when master nimbus lost ZK node
- Add exception catch when cgroups kill process
- Speed up reconnect to netty-server
- Share one task hearbeat thread for all tasks
- Quickly haltprocess when initialization failed.
- Check web-ui logview page size
- Add transaction programming mode
- Rewrite netty code, 1. use share boss/worker thread pool;2 async send batch tuples;3 single thread to do reconnect job;4 receive batch tuples
- Add metrics and statics
- Merge Alimama storm branch into this version, submit jar with -conf, -D, -lib
- add setting when supervisor has been shutdown, worker will shutdown automatically
- add LocalFristGrouping api
- enable cgroup for normal user
- Setting buffer size when upload jar
- Add lock between ZK watch and timer thread when refresh connection
- Enable nimbus monitor thread only when topology is running in cluster mode
- Fix exception when failed to read old assignment of ZK
- classloader fix when both parent and current classloader load the same class
- Fix log view null pointer exception
- switch apache thrift7 to storm thrift7
- set defatult acker number is 1
- add "spout.single.thread" setting
- make nimbus logview port different from supervisor's
- web ui can list all files of log's subdir
- Set gc dump dir as log's dir
- Support Aliyun Apsara/Hadoop Yarn
- Redesign Logview
- Kill old worker under the same port when worker is starting
- Add zk information/version information on UI
- Add nodeport information for dead task in nimbus
- Add interface to get values when spout doing ack
- Add timeout statics in bolt
- jstorm script return status
- Add logs when fail to deserialize tuple
- Skip sleep operation when max_pending is 1 and waiting ack
- Remove useless dependency
- Longer task timeout setting
- Add supervisor.use.ip setting
- Redirect supervisor out/err to /dev/null, redirect worker out/err to one file
- Fix kryo fail to deserialize object when enable classloader
- Fix fail to reassign dead task when worker number is less than topology apply
- Set samller jvm heap memory for jstorm-client
- Fix fail to set topology status as active when do rebalance operation twice at one time,
- Fix local mode bug under linux
- Fix average latency isn't accurate
- GC tuning.
- Add default kill function for AysncLoopRunnable
- Support LocalCluster/LocalDrpc mode, support debugging topology under local mode
- Support CGroups, assigning CPU in hardware level.
- Support simple logview
- Change SpoutExecutor's RotatingMap to TimeCacheMap, when putting too much timeout tuple is easy to cause deadlock in spout acker thread
- Tunning gc parameter, improve performance and avoid full GC
- Improve Topology's own gc priority, make it higher than JStorm system setting.
- Tuning Nimbus HA, switch nimbus faster, when occur nimbus failure.
- Fix bugs found by FindBugs tool.
- Revert Trident interface to 0.8.1, due to 0.8.1's trident interface's performance is better.
- Setting nimbus.task.timeout.secs as 60 to avoid nimbus doing assignment when task is under full gc.
- Setting default rpc framework as netty
- Tunning nimbus shutdown flow
- Tunning worker shutdown flow
- Add task heartbeat log
- Optimize Drpc/LocalDrpc source code.
- Move classloader to client jar. 14 Fix classloader fail to load anonymous class
- Web Ui display slave nimbus
- Add thrift max read buffer size
- Setting CPU slot base double
- Move Zk utility to jstorm-client-extension.jar
- Fix localOrShuffle null pointer
- Redirecting worker's System.out/System.err to file is configurable.
- Add new RPC frameworker JeroMq
- Fix Zk watcher miss problem
- Update sl4j 1.5.6 to 1.7.5
- Shutdown worker when occur exception in Smart thread
- Skip downloading useless topology in Supervisor
- Redownload the topology when failed to deserialize topology in Supervisor.
- Fix topology codeDir as resourceDir
- Catch error when normalize topology
- Add log when found one task is dead
- Add maven repository, JStorm is able to build outside of Alibaba
- Fix localOrShuffle null pointer exception
- Add statics counting for internal tuples in one worker
- Add thrift.close after download topology binary in Supervisor
- Application classloader. when Application jar is conflict with jstorm jar, please enable application classloader.
- Group Quato, Different group with different resource quato.
- Fix Rotation Map competition issue.
- Set default acker number as 0
- Set default spout/bolt number as 1
- Add log directory in log4j configuration file
- Add transaction example
- Fix UI showing wrong worker numbe in topology page
- Fix UI showing wrong latency in topology page
- Replace hardcode Integer convert with JStormUtils.parseInt
- Support string parse in Utils.getInt
- Remove useless dependency in pom.xml
- Support supervisor using IP or special hostname
- Add more details when no resource has been assigned to one new topology
- Replace normal thread with Smart thread
- Add gc details
- Code format
- Unify stormId and topologyId as topologyId
- Every nimbus will regist ip to ZK
In this version, it will follow storm 0.9.0 interface, so the application running on storm 0.9.0 can run in jstorm 0.9.0 without any change.
- provide nimbus HA. when the master nimbus shuts down, it will select another online nimbus to be the master. There is only one master nimbus online any time and the slave nimbuses just synchronouse the master's data.
- RPC through netty is stable, the sending speed is match with receiving speed.
- Assigning resource on four dimensions:cpu, mem, disk, net
- Application can use old assignment.
- Application can use user-define resource.
- Task can apply extra cpu slot or memory slot.
- Application can force tasks run on different supervisor or the same supervisor
In this version, it will follow storm 0.7.1 interface, so the topology running in storm 0.7.1 can run in jstorm without any change.
- Assign workers in balance
- add setting "zmq.max.queue.msg" for zeromq
- communication between worker and tasks without zeromq
- Add catch exception operation
- in supervisor SyncProcess/SyncSupervisor
- add catch exception and report_error in spout's open and bolt's prepare
- in all IO operation
- in all serialize/deserialize
- in all ZK operation
- in topology upload/download function
- during initialization zeromq
- do assignmen/reassignment operation in one thread to avoid competition
- redesign nimbus 's topology assign algorithm, make the logic simple much.
- redesign supervisor's sync assignment algorithm, make the logic simple much
- reduce zookeeper load
- redesign nimbus monitor logic, it will just scan tasks' hearbeat, frequency is 10s
- nimbus cancel watch on supervisor
- supervisor heartbeat frequence change to 10s
- supervisor syncSupervisor/syncProcess frequence change to 10s
- supervisor scan /$(ZKROOT)/assignment only once in one monitor loop
- task hearbeat change to 10s
- create task pid file before connection zk, this is very import when zk is unstable.
- reduce once memory copy when deserialize tuple, improve performance huge.
- split executor thread as two thread, one handing receive tuples, one sending tuples, improve performance much
- redeisign sample code, it will sampling every 5 seconds, not every 20 tuple once, improve performance much
- simplify the ack's logic, make acker more effeciency
- Communication between worker and tasks won't use zeromq, just memory share in process
- in worker's Drainer/virtualportdispatch thread, spout/bolt recv/send thread, the thread will sleep 1 ms when there is not tuple in one loop
- communication between worker and tasks without zeromq
- sampling frequence change to 5s, not every 20 tuple once.
- add IFailValueSpout interface
- Redesign sampling code, collection statics model become more common.
- Add sending/recving tps statics, statics is more precise.
- Atomatically do deactivate action when kill/rebalance topology, and the wait time is 2 * MSG_TIMEOUT
- fix nongrouping bug, random.nextInt will generate value less than 0.
- Sleep one setting time(default is 1 minute) after finish spout open, which is used to wait other task finish initialization.
- Add check component name when submit topology, forbidding the component which name start with "__"
- change the zk's node /$(ZKROOT)/storm to /$(ZKROOT)/topology
- abstract topology check logic from generating real topology function
- when supervisor is down and topology do rebalance, the alive task under down supervisor is unavailable.
- add close connection operation after finish download topology binary
- automatically create all local dirtorie, such as /$(LOCALDIR)/supervisor/localstate
- when killing worker, add "kill and sleep " operation before "kill -9" operation
- when generate real topology binary,
- configuration priority different.
component configuration > topology configuration > system configuration - skip the output stream which target component doesn't exist.
- skip the component whose parallism is 0.
- component's parallism is less than 0, throw exception.
- configuration priority different.
- skip ack/fail when inputstream setting is empty
- add topology name to the log
- fix ui select option error, default is 10 minutes
- supervisor can display all worker's status