[
https://issues.apache.org/jira/browse/STORM-561?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14531556#comment-14531556
]
ASF GitHub Bot commented on STORM-561:
--------------------------------------
Github user HeartSaVioR commented on a diff in the pull request:
https://github.com/apache/storm/pull/546#discussion_r29808769
--- Diff: external/flux/README.md ---
@@ -0,0 +1,834 @@
+# flux
+A framework for creating and deploying Apache Storm streaming computations
with less friction.
+
+## Definition
+**flux** |fləks| _noun_
+
+1. The action or process of flowing or flowing out
+2. Continuous change
+3. In physics, the rate of flow of a fluid, radiant energy, or particles
across a given area
+4. A substance mixed with a solid to lower its melting point
+
+## Rationale
+Bad things happen when configuration is hard-coded. No one should have to
recompile or repackage an application in
+order to change configuration.
+
+## About
+Flux is a framework and set of utilities that make defining and deploying
Apache Storm topologies less painful and
+deveoper-intensive.
+
+Have you ever found yourself repeating this pattern?:
+
+```java
+
+public static void main(String[] args) throws Exception {
+ // logic to determine if we're running locally or not...
+ // create necessary config options...
+ boolean runLocal = shouldRunLocal();
+ if(runLocal){
+ LocalCluster cluster = new LocalCluster();
+ cluster.submitTopology(name, conf, topology);
+ } else {
+ StormSubmitter.submitTopology(name, conf, topology);
+ }
+}
+```
+
+Wouldn't something like this be easier:
+
+```bash
+storm jar mytopology.jar org.apache.storm.flux.Flux --local config.yaml
+```
+
+or:
+
+```bash
+storm jar mytopology.jar org.apache.storm.flux.Flux --remote config.yaml
+```
+
+Another pain point often mentioned is the fact that the wiring for a
Topology graph is often tied up in Java code,
+and that any changes require recompilation and repackaging of the topology
jar file. Flux aims to alleviate that
+pain by allowing you to package all your Storm components in a single jar,
and use an external text file to define
+the layout and configuration of your topologies.
+
+## Features
+
+ * Easily configure and deploy Storm topologies (Both Storm core and
Microbatch API) without embedding configuration
+ in your topology code
+ * Support for existing topology code (see below)
+ * Define Storm Core API (Spouts/Bolts) using a flexible YAML DSL
+ * YAML DSL support for most Storm components (storm-kafka, storm-hdfs,
storm-hbase, etc.)
+ * Convenient support for multi-lang components
+ * External property substitution/filtering for easily switching between
configurations/environments (similar to Maven-style
+ `${variable.name}` substitution)
+
+## Usage
+
+To use Flux, add it as a dependency and package all your Storm components
in a fat jar, then create a YAML document
+to define your topology (see below for YAML configuration options).
+
+### Building from Source
+The easiest way to use Flux, is to add it as a Maven dependency in you
project as described below.
+
+If you would like to build Flux from source and run the unit/integration
tests, you will need the following installed
+on your system:
+
+* Python 2.6.x or later
+* Node.js 0.10.x or later
+
+#### Building with unit tests enabled:
+
+```
+mvn clean install
+```
+
+#### Building with unit tests disabled:
+If you would like to build Flux without installing Python or Node.js you
can simply skip the unit tests:
+
+```
+mvn clean install -DskipTests=true
+```
+
+Note that if you plan on using Flux to deploy topologies to a remote
cluster, you will still need to have Python
+installed since it is required by Apache Storm.
+
+
+#### Building with integration tests enabled:
+
+```
+mvn clean install -DskipIntegration=false
+```
+
+
+### Packaging with Maven
+To enable Flux for your Storm components, you need to add it as a
dependency such that it's included in the Storm
+topology jar. This can be accomplished with the Maven shade plugin
(preferred) or the Maven assembly plugin (not
+recommended).
+
+#### Flux Maven Dependency
+The current version of Flux is available in Maven Central at the following
coordinates:
+```xml
+<dependency>
+ <groupId>com.github.ptgoetz</groupId>
--- End diff --
It should be changed to fit to Storm external.
> Add ability to create topologies dynamically
> --------------------------------------------
>
> Key: STORM-561
> URL: https://issues.apache.org/jira/browse/STORM-561
> Project: Apache Storm
> Issue Type: Improvement
> Reporter: Nathan Leung
> Assignee: P. Taylor Goetz
> Original Estimate: 336h
> Remaining Estimate: 336h
>
> It would be nice if a storm topology could be built dynamically, instead of
> requiring a recompile to change parameters (e.g. number of workers, number of
> tasks, layout, etc).
> I would propose the following data structures for building core storm
> topologies. I haven't done a design for trident yet but the intention would
> be to add trident support when core storm support is complete (or in parallel
> if there are other people working on it):
> {code}
> // fields value and arguments are mutually exclusive
> class Argument {
> String argumentType; // Class used to lookup arguments in
> method/constructor
> String implementationType; // Class used to create this argument
> String value; // String used to construct this argument
> List<Argument> arguments; // arguments used to build this argument
> }
> class Dependency {
> String upstreamComponent; // name of upstream component
> String grouping;
> List<Argument> arguments; // arguments for the grouping
> }
> class StormSpout {
> String name;
> String klazz; // Class of this spout
> List <Argument> arguments;
> int numTasks;
> int numExecutors;
> }
> class StormBolt {
> String name;
> String klazz; // Class of this bolt
> List <Argument> arguments;
> int numTasks;
> int numExecutors;
> List<Dependency> dependencies;
> }
> class StormTopologyRepresentation {
> String name;
> List<StormSpout> spouts;
> List<StormBolt> bolts;
> Map config;
> int numWorkers;
> }
> {code}
> Topology creation will be built on top of the data structures above. The
> benefits:
> * Dependency free. Code to unmarshal from json, xml, etc, can be kept in
> extensions, or as examples, and users can write a different unmarshaller if
> they want to use a different text representation.
> * support arbitrary spout and bolts types
> * support of all groupings, streams, via reflections
> * ability to specify configuration map via config file
> * reification of spout / bolt / dependency arguments
> ** recursive argument reification for complex objects
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)