Repository: storm
Updated Branches:
  refs/heads/1.x-branch da35dabdd -> f0741f690


STORM-2098: DruidBeamBolt: Pass DruidConfig.Builder as constructor argument


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

Branch: refs/heads/1.x-branch
Commit: 8a0776d4baa0d2e7c3d44647eb909626c04bd60c
Parents: da35dab
Author: Manikumar Reddy O <manikumar.re...@gmail.com>
Authored: Mon Sep 19 10:52:38 2016 +0530
Committer: Manikumar Reddy O <manikumar.re...@gmail.com>
Committed: Mon Sep 19 10:52:38 2016 +0530

----------------------------------------------------------------------
 CHANGELOG.md                                                   | 1 +
 .../main/java/org/apache/storm/druid/bolt/DruidBeamBolt.java   | 4 ++--
 .../java/org/apache/storm/druid/SampleDruidBoltTopology.java   | 6 +++---
 3 files changed, 6 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/8a0776d4/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index 4ed3ea7..ab0bc6c 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
 ## 1.1.0
+ * STORM-2098: DruidBeamBolt: Pass DruidConfig.Builder as constructor argument
  * STORM-2092: optimize TridentKafkaState batch sending
  * STORM-1979: Storm Druid Connector implementation.
  * STORM-2057: Support JOIN statement in Storm SQL

http://git-wip-us.apache.org/repos/asf/storm/blob/8a0776d4/external/storm-druid/src/main/java/org/apache/storm/druid/bolt/DruidBeamBolt.java
----------------------------------------------------------------------
diff --git 
a/external/storm-druid/src/main/java/org/apache/storm/druid/bolt/DruidBeamBolt.java
 
b/external/storm-druid/src/main/java/org/apache/storm/druid/bolt/DruidBeamBolt.java
index 721eaa1..822b92c 100644
--- 
a/external/storm-druid/src/main/java/org/apache/storm/druid/bolt/DruidBeamBolt.java
+++ 
b/external/storm-druid/src/main/java/org/apache/storm/druid/bolt/DruidBeamBolt.java
@@ -55,9 +55,9 @@ public class DruidBeamBolt<E> extends BaseRichBolt {
     private Tranquilizer<E> tranquilizer = null;
     private ITupleDruidEventMapper<E> druidEventMapper = null;
 
-    public DruidBeamBolt(DruidBeamFactory<E> beamFactory, 
ITupleDruidEventMapper<E> druidEventMapper, DruidConfig druidConfig) {
+    public DruidBeamBolt(DruidBeamFactory<E> beamFactory, 
ITupleDruidEventMapper<E> druidEventMapper, DruidConfig.Builder 
druidConfigBuilder) {
         this.beamFactory = beamFactory;
-        this.druidConfig = druidConfig;
+        this.druidConfig = druidConfigBuilder.build();
         this.druidEventMapper = druidEventMapper;
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/8a0776d4/external/storm-druid/src/test/java/org/apache/storm/druid/SampleDruidBoltTopology.java
----------------------------------------------------------------------
diff --git 
a/external/storm-druid/src/test/java/org/apache/storm/druid/SampleDruidBoltTopology.java
 
b/external/storm-druid/src/test/java/org/apache/storm/druid/SampleDruidBoltTopology.java
index 99a6f67..88b2bf1 100644
--- 
a/external/storm-druid/src/test/java/org/apache/storm/druid/SampleDruidBoltTopology.java
+++ 
b/external/storm-druid/src/test/java/org/apache/storm/druid/SampleDruidBoltTopology.java
@@ -52,11 +52,11 @@ public class SampleDruidBoltTopology {
 
         topologyBuilder.setSpout("event-gen", new SimpleSpout(), 5);
         DruidBeamFactory druidBeamFactory = new SampleDruidBeamFactoryImpl(new 
HashMap<String, Object>());
-        DruidConfig druidConfig = 
DruidConfig.newBuilder().discardStreamId(DruidConfig.DEFAULT_DISCARD_STREAM_ID).build();
+        DruidConfig.Builder builder = 
DruidConfig.newBuilder().discardStreamId(DruidConfig.DEFAULT_DISCARD_STREAM_ID);
         ITupleDruidEventMapper<Map<String, Object>> eventMapper = new 
TupleDruidEventMapper<>(TupleDruidEventMapper.DEFAULT_FIELD_NAME);
-        DruidBeamBolt<Map<String, Object>> druidBolt = new 
DruidBeamBolt<Map<String, Object>>(druidBeamFactory, eventMapper, druidConfig);
+        DruidBeamBolt<Map<String, Object>> druidBolt = new 
DruidBeamBolt<Map<String, Object>>(druidBeamFactory, eventMapper, builder);
         topologyBuilder.setBolt("druid-bolt", 
druidBolt).shuffleGrouping("event-gen");
-        topologyBuilder.setBolt("printer-bolt", new 
PrinterBolt()).shuffleGrouping("druid-bolt" , druidConfig.getDiscardStreamId());
+        topologyBuilder.setBolt("printer-bolt", new 
PrinterBolt()).shuffleGrouping("druid-bolt" , 
DruidConfig.DEFAULT_DISCARD_STREAM_ID);
 
         Config conf = new Config();
         conf.setDebug(true);

Reply via email to