void flushAllWriters(boolean rollToNext) throws HiveWriter.CommitFailure, HiveWriter.TxnBatchFailure, HiveWriter.TxnFailure, InterruptedException { for (HiveWriter writer : allWriters.values()) { writer.flush(rollToNext); } }
@Override public void cleanup() { for (Entry<HiveEndPoint, HiveWriter> entry : allWriters.entrySet()) { try { HiveWriter w = entry.getValue(); LOG.info("Flushing writer to {}", w); w.flush(false); LOG.info("Closing writer to {}", w); w.close(); } catch (Exception ex) { LOG.warn("Error while closing writer to " + entry.getKey() + ". Exception follows.", ex); if (ex instanceof InterruptedException) { Thread.currentThread().interrupt(); } } } ExecutorService toShutdown[] = {callTimeoutPool}; for (ExecutorService execService : toShutdown) { execService.shutdown(); try { while (!execService.isTerminated()) { execService.awaitTermination(options.getCallTimeOut(), TimeUnit.MILLISECONDS); } } catch (InterruptedException ex) { LOG.warn("shutdown interrupted on " + execService, ex); } } callTimeoutPool = null; super.cleanup(); LOG.info("Hive Bolt stopped"); }
private void flushAllWriters() throws HiveWriter.CommitFailure, HiveWriter.TxnBatchFailure, HiveWriter.TxnFailure, InterruptedException { for (HiveWriter writer : allWriters.values()) { writer.flush(true); } }
@Override public void execute(Tuple tuple) { try { boolean forceFlush = false; if (TupleUtils.isTick(tuple)) { LOG.debug( "TICK received! current batch status [{}/{}]", tupleBatch.size(), options.getBatchSize()); collector.ack(tuple); forceFlush = true; } else { List<String> partitionVals = options.getMapper().mapPartitions(tuple); HiveEndPoint endPoint = HiveUtils.makeEndPoint(partitionVals, options); HiveWriter writer = getOrCreateWriter(endPoint); if (timeToSendHeartBeat.compareAndSet(true, false)) { enableHeartBeatOnAllWriters(); } writer.write(options.getMapper().mapRecord(tuple)); tupleBatch.add(tuple); if (tupleBatch.size() >= options.getBatchSize()) forceFlush = true; } if (forceFlush && !tupleBatch.isEmpty()) { flushAllWriters(true); LOG.info("acknowledging tuples after writers flushed "); for (Tuple t : tupleBatch) collector.ack(t); tupleBatch.clear(); } } catch (Exception e) { this.collector.reportError(e); collector.fail(tuple); try { flushAndCloseWriters(); LOG.info("acknowledging tuples after writers flushed and closed"); for (Tuple t : tupleBatch) collector.ack(t); tupleBatch.clear(); } catch (Exception e1) { // If flushAndClose fails assume tuples are lost, do not ack LOG.warn("Error while flushing and closing writers, tuples will NOT be acknowledged"); for (Tuple t : tupleBatch) collector.fail(t); tupleBatch.clear(); } } }
@Override public void execute(Tuple tuple) { try { List<String> partitionVals = options.getMapper().mapPartitions(tuple); HiveEndPoint endPoint = HiveUtils.makeEndPoint(partitionVals, options); HiveWriter writer = getOrCreateWriter(endPoint); if (timeToSendHeartBeat.compareAndSet(true, false)) { enableHeartBeatOnAllWriters(); } writer.write(options.getMapper().mapRecord(tuple)); currentBatchSize++; if (currentBatchSize >= options.getBatchSize()) { flushAllWriters(); currentBatchSize = 0; } collector.ack(tuple); } catch (Exception e) { this.collector.reportError(e); collector.fail(tuple); flushAndCloseWriters(); } }
private void enableHeartBeatOnAllWriters() { for (HiveWriter writer : allWriters.values()) { writer.setHeartBeatNeeded(); } }