You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by pt...@apache.org on 2013/12/07 07:09:57 UTC

[01/12] git commit: Add client/server thread config to fix netty transport unit tests.

Updated Branches:
  refs/heads/master 66a397368 -> 1bcc169f5


Add client/server thread config to fix netty transport unit tests.


Project: http://git-wip-us.apache.org/repos/asf/incubator-storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-storm/commit/66628504
Tree: http://git-wip-us.apache.org/repos/asf/incubator-storm/tree/66628504
Diff: http://git-wip-us.apache.org/repos/asf/incubator-storm/diff/66628504

Branch: refs/heads/master
Commit: 66628504e9e706386a5d53cee3b0afc9d90cc757
Parents: 66a3973
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Tue Nov 26 15:07:32 2013 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Tue Nov 26 15:07:32 2013 -0500

----------------------------------------------------------------------
 .../storm/messaging/netty_integration_test.clj  |  5 ++++-
 .../storm/messaging/netty_unit_test.clj         | 20 ++++++++++++++++----
 2 files changed, 20 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/66628504/storm-netty/test/clj/backtype/storm/messaging/netty_integration_test.clj
----------------------------------------------------------------------
diff --git a/storm-netty/test/clj/backtype/storm/messaging/netty_integration_test.clj b/storm-netty/test/clj/backtype/storm/messaging/netty_integration_test.clj
index e9338e7..eefcb48 100644
--- a/storm-netty/test/clj/backtype/storm/messaging/netty_integration_test.clj
+++ b/storm-netty/test/clj/backtype/storm/messaging/netty_integration_test.clj
@@ -13,7 +13,10 @@
                                                     STORM-MESSAGING-NETTY-BUFFER-SIZE 1024000
                                                     STORM-MESSAGING-NETTY-MAX-RETRIES 10
                                                     STORM-MESSAGING-NETTY-MIN-SLEEP-MS 1000 
-                                                    STORM-MESSAGING-NETTY-MAX-SLEEP-MS 5000}]
+                                                    STORM-MESSAGING-NETTY-MAX-SLEEP-MS 5000
+                                                    STORM-MESSAGING-NETTY-CLIENT-WORKER-THREADS 1
+                                                    STORM-MESSAGING-NETTY-SERVER-WORKER-THREADS 1
+                                                    }]
     (let [topology (thrift/mk-topology
                      {"1" (thrift/mk-spout-spec (TestWordSpout. true) :parallelism-hint 4)}
                      {"2" (thrift/mk-bolt-spec {"1" :shuffle} (TestGlobalCount.)

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/66628504/storm-netty/test/clj/backtype/storm/messaging/netty_unit_test.clj
----------------------------------------------------------------------
diff --git a/storm-netty/test/clj/backtype/storm/messaging/netty_unit_test.clj b/storm-netty/test/clj/backtype/storm/messaging/netty_unit_test.clj
index 441360a..12ebe5d 100644
--- a/storm-netty/test/clj/backtype/storm/messaging/netty_unit_test.clj
+++ b/storm-netty/test/clj/backtype/storm/messaging/netty_unit_test.clj
@@ -14,7 +14,10 @@
                     STORM-MESSAGING-NETTY-BUFFER-SIZE 1024
                     STORM-MESSAGING-NETTY-MAX-RETRIES 10
                     STORM-MESSAGING-NETTY-MIN-SLEEP-MS 1000 
-                    STORM-MESSAGING-NETTY-MAX-SLEEP-MS 5000 }
+                    STORM-MESSAGING-NETTY-MAX-SLEEP-MS 5000
+                    STORM-MESSAGING-NETTY-SERVER-WORKER-THREADS 1
+                    STORM-MESSAGING-NETTY-CLIENT-WORKER-THREADS 1
+                    }
         context (TransportFactory/makeContext storm-conf)
         server (.bind context nil port)
         client (.connect context nil "localhost" port)
@@ -32,7 +35,10 @@
                     STORM-MESSAGING-NETTY-BUFFER-SIZE 102400
                     STORM-MESSAGING-NETTY-MAX-RETRIES 10
                     STORM-MESSAGING-NETTY-MIN-SLEEP-MS 1000 
-                    STORM-MESSAGING-NETTY-MAX-SLEEP-MS 5000 }
+                    STORM-MESSAGING-NETTY-MAX-SLEEP-MS 5000
+                    STORM-MESSAGING-NETTY-SERVER-WORKER-THREADS 1
+                    STORM-MESSAGING-NETTY-CLIENT-WORKER-THREADS 1
+                    }
         context (TransportFactory/makeContext storm-conf)
         server (.bind context nil port)
         client (.connect context nil "localhost" port)
@@ -50,7 +56,10 @@
                     STORM-MESSAGING-NETTY-BUFFER-SIZE 1024
                     STORM-MESSAGING-NETTY-MAX-RETRIES 10
                     STORM-MESSAGING-NETTY-MIN-SLEEP-MS 1000 
-                    STORM-MESSAGING-NETTY-MAX-SLEEP-MS 5000 }
+                    STORM-MESSAGING-NETTY-MAX-SLEEP-MS 5000
+                    STORM-MESSAGING-NETTY-SERVER-WORKER-THREADS 1
+                    STORM-MESSAGING-NETTY-CLIENT-WORKER-THREADS 1
+                    }
         context (TransportFactory/makeContext storm-conf)
         client (.connect context nil "localhost" port)
         _ (.send client task (.getBytes req_msg))
@@ -68,7 +77,10 @@
                     STORM-MESSAGING-NETTY-BUFFER-SIZE 1024000
                     STORM-MESSAGING-NETTY-MAX-RETRIES 10
                     STORM-MESSAGING-NETTY-MIN-SLEEP-MS 1000 
-                    STORM-MESSAGING-NETTY-MAX-SLEEP-MS 5000}
+                    STORM-MESSAGING-NETTY-MAX-SLEEP-MS 5000
+                    STORM-MESSAGING-NETTY-SERVER-WORKER-THREADS 1
+                    STORM-MESSAGING-NETTY-CLIENT-WORKER-THREADS 1
+                    }
         context (TransportFactory/makeContext storm-conf)
         server (.bind context nil port)
         client (.connect context nil "localhost" port)]


[11/12] git commit: update build to force Java 1.6 compatibility.

Posted by pt...@apache.org.
update build to force Java 1.6 compatibility.


Project: http://git-wip-us.apache.org/repos/asf/incubator-storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-storm/commit/b201033f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-storm/tree/b201033f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-storm/diff/b201033f

Branch: refs/heads/master
Commit: b201033f5a0cf9a680f2b71e395756ce552704a0
Parents: 061ee2e
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Thu Dec 5 10:05:57 2013 -0800
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Thu Dec 5 10:05:57 2013 -0800

----------------------------------------------------------------------
 storm-console-logging/project.clj | 2 +-
 storm-core/project.clj            | 2 +-
 storm-netty/project.clj           | 1 +
 3 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/b201033f/storm-console-logging/project.clj
----------------------------------------------------------------------
diff --git a/storm-console-logging/project.clj b/storm-console-logging/project.clj
index 56d71c7..7b4322b 100644
--- a/storm-console-logging/project.clj
+++ b/storm-console-logging/project.clj
@@ -4,7 +4,7 @@
 (defproject storm/storm-console-logging VERSION
   :resource-paths ["logback"]
   :target-path "target"
-
+  :javac-options ["-target" "1.6" "-source" "1.6"]
   :profiles {:release {}
              }
 

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/b201033f/storm-core/project.clj
----------------------------------------------------------------------
diff --git a/storm-core/project.clj b/storm-core/project.clj
index e274e66..0eaa6a3 100644
--- a/storm-core/project.clj
+++ b/storm-core/project.clj
@@ -34,7 +34,7 @@
   :test-paths ["test/clj"]
   :resource-paths ["../conf"]
   :target-path "target"
-
+  :javac-options ["-target" "1.6" "-source" "1.6"]
   :profiles {:dev {:resource-paths ["src/dev"]
                    :dependencies [[org.mockito/mockito-all "1.9.5"]]}
              :release {}

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/b201033f/storm-netty/project.clj
----------------------------------------------------------------------
diff --git a/storm-netty/project.clj b/storm-netty/project.clj
index 869179c..24905bf 100644
--- a/storm-netty/project.clj
+++ b/storm-netty/project.clj
@@ -9,4 +9,5 @@
   :profiles {:release {}}
   :jvm-opts ["-Djava.library.path=/usr/local/lib:/opt/local/lib:/usr/lib"]
   :target-path "target"
+  :javac-options ["-target" "1.6" "-source" "1.6"]
   :aot :all))


[08/12] git commit: Merge branch 'release-cleanup'

Posted by pt...@apache.org.
Merge branch 'release-cleanup'


Project: http://git-wip-us.apache.org/repos/asf/incubator-storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-storm/commit/5890ee4e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-storm/tree/5890ee4e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-storm/diff/5890ee4e

Branch: refs/heads/master
Commit: 5890ee4eb2bdbbc2d29c14e973bc9b5a4acdcefe
Parents: 9aed0de eb45818
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Wed Dec 4 15:15:51 2013 -0800
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Wed Dec 4 15:15:51 2013 -0800

----------------------------------------------------------------------
 .gitignore           | 1 +
 bin/build_release.sh | 8 ++++++--
 bin/storm            | 2 +-
 3 files changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------



[06/12] git commit: Ensure we don't overflow the backoff value.

Posted by pt...@apache.org.
Ensure we don't overflow the backoff value.

The first attempt to fix this (213102b36f890) did not correctly address
the issue.  The 32 bit signed integer frequently overflows, resulting in
a bad value for Random.nextInt().

The default for storm.messaging.netty.max_retries is now 30 (instead of
100), and there is an upper limit of 30 for max_retries.

I also did a whitespace cleanup.


Project: http://git-wip-us.apache.org/repos/asf/incubator-storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-storm/commit/c638db0e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-storm/tree/c638db0e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-storm/diff/c638db0e

Branch: refs/heads/master
Commit: c638db0e88e3c56f808c8a76a88f94d7bf1988c4
Parents: 4e19589
Author: Brenden Matthews <br...@diddyinc.com>
Authored: Wed Oct 30 09:41:13 2013 -0700
Committer: Brenden Matthews <br...@diddyinc.com>
Committed: Wed Dec 4 14:30:52 2013 -0800

----------------------------------------------------------------------
 conf/defaults.yaml                              |  2 +-
 .../backtype/storm/messaging/netty/Client.java  | 58 ++++++++++----------
 2 files changed, 29 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/c638db0e/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index 08c7889..a5b31f4 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -86,7 +86,7 @@ zmq.hwm: 0
 storm.messaging.netty.server_worker_threads: 1
 storm.messaging.netty.client_worker_threads: 1
 storm.messaging.netty.buffer_size: 5242880 #5MB buffer
-storm.messaging.netty.max_retries: 100
+storm.messaging.netty.max_retries: 30
 storm.messaging.netty.max_wait_ms: 1000
 storm.messaging.netty.min_wait_ms: 100
 

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/c638db0e/storm-netty/src/jvm/backtype/storm/messaging/netty/Client.java
----------------------------------------------------------------------
diff --git a/storm-netty/src/jvm/backtype/storm/messaging/netty/Client.java b/storm-netty/src/jvm/backtype/storm/messaging/netty/Client.java
index 00431d4..91e4bd4 100644
--- a/storm-netty/src/jvm/backtype/storm/messaging/netty/Client.java
+++ b/storm-netty/src/jvm/backtype/storm/messaging/netty/Client.java
@@ -23,14 +23,14 @@ import backtype.storm.utils.Utils;
 
 class Client implements IConnection {
     private static final Logger LOG = LoggerFactory.getLogger(Client.class);
-    private final int max_retries; 
-    private final int base_sleep_ms; 
-    private final int max_sleep_ms; 
+    private final int max_retries;
+    private final int base_sleep_ms;
+    private final int max_sleep_ms;
     private LinkedBlockingQueue<Object> message_queue; //entry should either be TaskMessage or ControlMessage
     private AtomicReference<Channel> channelRef;
     private final ClientBootstrap bootstrap;
     private InetSocketAddress remote_addr;
-    private AtomicInteger retries; 
+    private AtomicInteger retries;
     private final Random random = new Random();
     private final ChannelFactory factory;
     private final int buffer_size;
@@ -38,14 +38,14 @@ class Client implements IConnection {
 
     @SuppressWarnings("rawtypes")
     Client(Map storm_conf, String host, int port) {
-        message_queue = new LinkedBlockingQueue<Object>(); 
+        message_queue = new LinkedBlockingQueue<Object>();
         retries = new AtomicInteger(0);
         channelRef = new AtomicReference<Channel>(null);
         being_closed = new AtomicBoolean(false);
 
-        // Configure 
+        // Configure
         buffer_size = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_BUFFER_SIZE));
-        max_retries = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_MAX_RETRIES));
+        max_retries = Math.min(30, Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_MAX_RETRIES)));
         base_sleep_ms = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_MIN_SLEEP_MS));
         max_sleep_ms = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_MAX_SLEEP_MS));
         int maxWorkers = Utils.getInt(storm_conf.get(Config.STORM_MESSAGING_NETTY_CLIENT_WORKER_THREADS));
@@ -74,9 +74,9 @@ class Client implements IConnection {
     void reconnect() {
         try {
             int tried_count = retries.incrementAndGet();
-            if (tried_count < max_retries) {
+            if (tried_count <= max_retries) {
                 Thread.sleep(getSleepTimeMs());
-                LOG.info("Reconnect ... [{}]", tried_count);   
+                LOG.info("Reconnect ... [{}]", tried_count);
                 bootstrap.connect(remote_addr);
                 LOG.debug("connection started...");
             } else {
@@ -85,7 +85,7 @@ class Client implements IConnection {
             }
         } catch (InterruptedException e) {
             LOG.warn("connection failed", e);
-        } 
+        }
     }
 
     /**
@@ -93,19 +93,17 @@ class Client implements IConnection {
      */
     private int getSleepTimeMs()
     {
-        int backoff = 1 << Math.max(1, retries.get());
+        int backoff = 1 << retries.get();
         int sleepMs = base_sleep_ms * Math.max(1, random.nextInt(backoff));
         if ( sleepMs > max_sleep_ms )
             sleepMs = max_sleep_ms;
-        if ( sleepMs < base_sleep_ms )
-          sleepMs = base_sleep_ms;
         return sleepMs;
     }
 
     /**
-     * Enqueue a task message to be sent to server 
+     * Enqueue a task message to be sent to server
      */
-    public void send(int task, byte[] message) {        
+    public void send(int task, byte[] message) {
         //throw exception if the client is being closed
         if (being_closed.get()) {
             throw new RuntimeException("Client is being closed, and does not take requests any more");
@@ -128,43 +126,43 @@ class Client implements IConnection {
         MessageBatch batch = new MessageBatch(buffer_size);
         Object msg = message_queue.take();
         batch.add(msg);
-        
+
         //we will discard any message after CLOSE
-        if (msg==ControlMessage.CLOSE_MESSAGE) 
+        if (msg==ControlMessage.CLOSE_MESSAGE)
             return batch;
-        
+
         while (!batch.isFull()) {
             //peek the next message
             msg = message_queue.peek();
             //no more messages
             if (msg == null) break;
-            
+
             //we will discard any message after CLOSE
             if (msg==ControlMessage.CLOSE_MESSAGE) {
                 message_queue.take();
                 batch.add(msg);
                 break;
             }
-            
+
             //try to add this msg into batch
             if (!batch.tryAdd((TaskMessage) msg))
                 break;
-            
+
             //remove this message
             message_queue.take();
         }
 
         return batch;
     }
-    
+
     /**
      * gracefully close this client.
-     * 
+     *
      * We will send all existing requests, and then invoke close_n_release() method
      */
     public synchronized void close() {
-        if (!being_closed.get()) {  
-            //enqueue a CLOSE message so that shutdown() will be invoked 
+        if (!being_closed.get()) {
+            //enqueue a CLOSE message so that shutdown() will be invoked
             try {
                 message_queue.put(ControlMessage.CLOSE_MESSAGE);
                 being_closed.set(true);
@@ -178,10 +176,10 @@ class Client implements IConnection {
      * close_n_release() is invoked after all messages have been sent.
      */
     void  close_n_release() {
-        if (channelRef.get() != null) 
+        if (channelRef.get() != null)
             channelRef.get().close().awaitUninterruptibly();
 
-        //we need to release resources 
+        //we need to release resources
         new Thread(new Runnable() {
             @Override
             public void run() {
@@ -194,10 +192,10 @@ class Client implements IConnection {
     }
 
     void setChannel(Channel channel) {
-        channelRef.set(channel); 
-        //reset retries   
+        channelRef.set(channel);
+        //reset retries
         if (channel != null)
-            retries.set(0); 
+            retries.set(0);
     }
 
 }


[10/12] git commit: update changelog for 0.9.0 release

Posted by pt...@apache.org.
update changelog for 0.9.0 release


Project: http://git-wip-us.apache.org/repos/asf/incubator-storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-storm/commit/061ee2e7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-storm/tree/061ee2e7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-storm/diff/061ee2e7

Branch: refs/heads/master
Commit: 061ee2e7ca6ed89dbf0522ba3b2ccee6558c4105
Parents: 6466d04
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Wed Dec 4 15:36:07 2013 -0800
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Wed Dec 4 15:36:07 2013 -0800

----------------------------------------------------------------------
 CHANGELOG.md | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/061ee2e7/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 4e6b579..02b1e71 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,6 @@
-## 0.9.0-rc4 (Unreleased)
+## 0.9.0
+* Fixed a netty client issue where sleep times for reconnection could be negative (thanks brndnmtthws)
+* Fixed an issue that would cause storm-netty unit tests to fail
 
 ## 0.9.0-rc3
 * Added configuration to limit ShellBolt internal _pendingWrites queue length (thanks xiaokang)


[03/12] git commit: fix download URL in the 'storm' command

Posted by pt...@apache.org.
fix download URL in the 'storm' command


Project: http://git-wip-us.apache.org/repos/asf/incubator-storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-storm/commit/df73338e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-storm/tree/df73338e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-storm/diff/df73338e

Branch: refs/heads/master
Commit: df73338e7bb4871277ec68779f7bb632d471a229
Parents: 6dfc237
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Wed Nov 27 12:39:09 2013 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Wed Nov 27 12:39:09 2013 -0500

----------------------------------------------------------------------
 bin/storm | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/df73338e/bin/storm
----------------------------------------------------------------------
diff --git a/bin/storm b/bin/storm
index 444fb8b..fc08a45 100755
--- a/bin/storm
+++ b/bin/storm
@@ -39,7 +39,7 @@ def get_config_opts():
 if not os.path.exists(STORM_DIR + "/RELEASE"):
     print "******************************************"
     print "The storm client can only be run from within a release. You appear to be trying to run the client from a checkout of Storm's source code."
-    print "\nYou can download a Storm release at https://github.com/nathanmarz/storm/downloads"
+    print "\nYou can download a Storm release at http://storm-project.net/downloads.html"
     print "******************************************"
     sys.exit(1)  
 


[07/12] git commit: Merge pull request #726 from brndnmtthws/nm

Posted by pt...@apache.org.
Merge pull request #726 from brndnmtthws/nm

Ensure we don't overflow the backoff value.

Project: http://git-wip-us.apache.org/repos/asf/incubator-storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-storm/commit/9aed0ded
Tree: http://git-wip-us.apache.org/repos/asf/incubator-storm/tree/9aed0ded
Diff: http://git-wip-us.apache.org/repos/asf/incubator-storm/diff/9aed0ded

Branch: refs/heads/master
Commit: 9aed0dede671688355d0a866b23167aadab47397
Parents: 4e19589 c638db0
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Wed Dec 4 15:08:06 2013 -0800
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Wed Dec 4 15:08:06 2013 -0800

----------------------------------------------------------------------
 conf/defaults.yaml                              |  2 +-
 .../backtype/storm/messaging/netty/Client.java  | 58 ++++++++++----------
 2 files changed, 29 insertions(+), 31 deletions(-)
----------------------------------------------------------------------



[12/12] git commit: bump version, update changelog for 0.9.0.1 release.

Posted by pt...@apache.org.
bump version, update changelog for 0.9.0.1 release.


Project: http://git-wip-us.apache.org/repos/asf/incubator-storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-storm/commit/1bcc169f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-storm/tree/1bcc169f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-storm/diff/1bcc169f

Branch: refs/heads/master
Commit: 1bcc169f5096e03a4ae117efc65c0f9bcfa2fa22
Parents: b201033
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Thu Dec 5 10:33:36 2013 -0800
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Thu Dec 5 10:33:36 2013 -0800

----------------------------------------------------------------------
 CHANGELOG.md | 3 +++
 VERSION      | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/1bcc169f/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 02b1e71..0982606 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,3 +1,6 @@
+## 0.9.0.1
+* Update build configuration to force compatibility with Java 1.6
+
 ## 0.9.0
 * Fixed a netty client issue where sleep times for reconnection could be negative (thanks brndnmtthws)
 * Fixed an issue that would cause storm-netty unit tests to fail

http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/1bcc169f/VERSION
----------------------------------------------------------------------
diff --git a/VERSION b/VERSION
index ac39a10..15e36e6 100644
--- a/VERSION
+++ b/VERSION
@@ -1 +1 @@
-0.9.0
+0.9.0.1


[04/12] git commit: add *.tar.gz to .gitignore

Posted by pt...@apache.org.
add *.tar.gz to .gitignore


Project: http://git-wip-us.apache.org/repos/asf/incubator-storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-storm/commit/eb45818f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-storm/tree/eb45818f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-storm/diff/eb45818f

Branch: refs/heads/master
Commit: eb45818f453fbbcef610df757edc21ddbc67f5de
Parents: df73338
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Wed Nov 27 13:05:49 2013 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Wed Nov 27 13:05:49 2013 -0500

----------------------------------------------------------------------
 .gitignore | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/eb45818f/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 8b9182d..7faaac8 100644
--- a/.gitignore
+++ b/.gitignore
@@ -20,6 +20,7 @@ NANNY
 .lein-failures
 _release
 *.zip
+*.tar.gz
 .lein-deps-sum
 *.iml
 target


[05/12] git commit: Merge pull request #755 from ptgoetz/fix-netty-tests

Posted by pt...@apache.org.
Merge pull request #755 from ptgoetz/fix-netty-tests

Fix netty transport unit tests

Project: http://git-wip-us.apache.org/repos/asf/incubator-storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-storm/commit/4e195891
Tree: http://git-wip-us.apache.org/repos/asf/incubator-storm/tree/4e195891
Diff: http://git-wip-us.apache.org/repos/asf/incubator-storm/diff/4e195891

Branch: refs/heads/master
Commit: 4e195891b25becc9995dce0f4ba64b061c540459
Parents: 66a3973 6662850
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Mon Dec 2 10:40:41 2013 -0800
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Mon Dec 2 10:40:41 2013 -0800

----------------------------------------------------------------------
 .../storm/messaging/netty_integration_test.clj  |  5 ++++-
 .../storm/messaging/netty_unit_test.clj         | 20 ++++++++++++++++----
 2 files changed, 20 insertions(+), 5 deletions(-)
----------------------------------------------------------------------



[09/12] git commit: bump version for 0.9.0 release

Posted by pt...@apache.org.
bump version for 0.9.0 release


Project: http://git-wip-us.apache.org/repos/asf/incubator-storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-storm/commit/6466d044
Tree: http://git-wip-us.apache.org/repos/asf/incubator-storm/tree/6466d044
Diff: http://git-wip-us.apache.org/repos/asf/incubator-storm/diff/6466d044

Branch: refs/heads/master
Commit: 6466d0446beec82f2c8b03390eb350251a740abe
Parents: 5890ee4
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Wed Dec 4 15:19:30 2013 -0800
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Wed Dec 4 15:19:30 2013 -0800

----------------------------------------------------------------------
 VERSION | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/6466d044/VERSION
----------------------------------------------------------------------
diff --git a/VERSION b/VERSION
index 94f6550..ac39a10 100644
--- a/VERSION
+++ b/VERSION
@@ -1 +1 @@
-0.9.0-rc3
+0.9.0


[02/12] git commit: add generation of a .tar.gz to release script

Posted by pt...@apache.org.
add generation of a .tar.gz to release script


Project: http://git-wip-us.apache.org/repos/asf/incubator-storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-storm/commit/6dfc2377
Tree: http://git-wip-us.apache.org/repos/asf/incubator-storm/tree/6dfc2377
Diff: http://git-wip-us.apache.org/repos/asf/incubator-storm/diff/6dfc2377

Branch: refs/heads/master
Commit: 6dfc2377648962410d9bbcbcf13fa3c56ed8b09d
Parents: 66a3973
Author: P. Taylor Goetz <pt...@gmail.com>
Authored: Wed Nov 27 12:33:33 2013 -0500
Committer: P. Taylor Goetz <pt...@gmail.com>
Committed: Wed Nov 27 12:33:33 2013 -0500

----------------------------------------------------------------------
 bin/build_release.sh | 8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-storm/blob/6dfc2377/bin/build_release.sh
----------------------------------------------------------------------
diff --git a/bin/build_release.sh b/bin/build_release.sh
index 723b2f4..b6f3cab 100644
--- a/bin/build_release.sh
+++ b/bin/build_release.sh
@@ -13,7 +13,8 @@ echo Making release $RELEASE
 DIR=`pwd`/_release/storm-$RELEASE
 
 rm -rf _release
-rm -f *.zip 
+rm -f *.zip
+rm -f *.tar.gz
 $LEIN pom || exit 1
 mkdir -p $DIR/lib
 
@@ -56,7 +57,10 @@ cp LICENSE.html $DIR/
 
 cd _release
 zip -r storm-$RELEASE.zip *
+mv storm-*.zip ../
+tar -cvzf ../storm-$RELEASE.tar.gz ./
+
 cd ..
-mv _release/storm-*.zip .
+
 rm -rf _release