Github user dossett commented on a diff in the pull request: https://github.com/apache/storm/pull/664#discussion_r37241514 --- Diff: external/storm-hdfs/src/main/java/org/apache/storm/hdfs/bolt/HdfsBolt.java --- @@ -88,35 +99,94 @@ public void doPrepare(Map conf, TopologyContext topologyContext, OutputCollector @Override public void execute(Tuple tuple) { - try { - byte[] bytes = this.format.format(tuple); - synchronized (this.writeLock) { - out.write(bytes); - this.offset += bytes.length; - - if (this.syncPolicy.mark(tuple, this.offset)) { - if (this.out instanceof HdfsDataOutputStream) { - ((HdfsDataOutputStream) this.out).hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH)); - } else { - this.out.hsync(); - } - this.syncPolicy.reset(); + boolean forceRotate = false; + synchronized (this.writeLock) { + boolean forceSync = false; + if (TupleUtils.isTick(tuple)) { + LOG.debug("TICK! forcing a file system flush"); + forceSync = true; + } + else { + try { + writeAndAddTuple(tuple); + } catch (IOException e) { + //If the write failed, try to sync anything already written + LOG.info("Tuple failed to write, forcing a flush of existing data."); + this.collector.reportError(e); + forceSync = true; + this.collector.fail(tuple); } } - this.collector.ack(tuple); + if (this.syncPolicy.mark(tuple, this.offset) || forceSync) { + try { + syncAndAckTuples(); + } catch (IOException e) { + LOG.warn("Data could not be synced to filesystem, failing this batch of tuples"); + this.collector.reportError(e); + //Force rotation to get a new file handle + forceRotate = true; + for (Tuple t : tupleBatch) + this.collector.fail(t); + tupleBatch.clear(); + } + } + } - if(this.rotationPolicy.mark(tuple, this.offset)){ - rotateOutputFile(); // synchronized - this.offset = 0; - this.rotationPolicy.reset(); + if(this.rotationPolicy.mark(tuple, this.offset) || forceRotate) { + try { + rotateAndReset(); + } catch (IOException e) { + this.collector.reportError(e); + LOG.warn("File could not be rotated"); + //At this point there is nothing to do. In all likelihood any filesystem operations will fail. + //The next tuple will almost certainly fail to write and/or sync, which force a rotation. That + //will give rotateAndReset() a chance to work which includes creating a fresh file handle. } - } catch (IOException e) { - this.collector.reportError(e); - this.collector.fail(tuple); } } + private void rotateAndReset() throws IOException { --- End diff -- I suppose it's possible that empty files would be created, but if a filesystem is in such a bad state I would expect creating new files to fail as well. Killing the task after enough failures is an interesting idea... what default value for # of attempts would you consider reasonable?
--- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---