[ 
https://issues.apache.org/jira/browse/STORM-960?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14646317#comment-14646317
 ] 

ASF GitHub Bot commented on STORM-960:
--------------------------------------

Github user dossett commented on a diff in the pull request:

    https://github.com/apache/storm/pull/653#discussion_r35777089
  
    --- Diff: 
external/storm-hive/src/main/java/org/apache/storm/hive/bolt/HiveBolt.java ---
    @@ -43,23 +43,25 @@
     import java.util.concurrent.Executors;
     import java.util.concurrent.TimeUnit;
     import java.util.concurrent.atomic.AtomicBoolean;
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.LinkedBlockingQueue;
     import java.io.IOException;
     
     public class HiveBolt extends  BaseRichBolt {
         private static final Logger LOG = 
LoggerFactory.getLogger(HiveBolt.class);
         private OutputCollector collector;
         private HiveOptions options;
    -    private Integer currentBatchSize;
         private ExecutorService callTimeoutPool;
         private transient Timer heartBeatTimer;
         private Boolean kerberosEnabled = false;
         private AtomicBoolean timeToSendHeartBeat = new AtomicBoolean(false);
         private UserGroupInformation ugi = null;
         HashMap<HiveEndPoint, HiveWriter> allWriters;
    +    private BlockingQueue<Tuple> tupleBatch;
     
         public HiveBolt(HiveOptions options) {
             this.options = options;
    -        this.currentBatchSize = 0;
    +        tupleBatch = new LinkedBlockingQueue<Tuple>();
    --- End diff --
    
    I was trying to guard against any concurrency issues with that data 
structure.  I still have not mastered how storm components parallelize, so 
maybe this isn't needed?


> Hive-Bolt can lose tuples when flushing data
> --------------------------------------------
>
>                 Key: STORM-960
>                 URL: https://issues.apache.org/jira/browse/STORM-960
>             Project: Apache Storm
>          Issue Type: Improvement
>          Components: external
>            Reporter: Aaron Dossett
>            Assignee: Aaron Dossett
>            Priority: Minor
>
> In HiveBolt's execute method tuples are ack'd as they are received.  When a 
> batchsize of tuples has been received, the writers are flushed.  However, if 
> the flush fails only the most recent tuple will be marked as failed.  All 
> prior tuples will already have been ack'd.  This creates a window for data 
> loss.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to