@Override public boolean isSpeculativeAttemptSafe() { if (this.writer instanceof SpeculativeAttemptAwareConstruct) { return ((SpeculativeAttemptAwareConstruct)this.writer).isSpeculativeAttemptSafe(); } return false; }
private static void executeCommitSequence(List<CommitStep> steps) throws IOException { for (CommitStep step : steps) { step.execute(); } }
public static Builder newBuilder() { return new Builder(); } }
@BeforeClass public void setUp() throws IOException { FileSystem fs = FileSystem.getLocal(new Configuration()); this.store = new FsCommitSequenceStore(fs, new Path("commit-sequence-store-test")); State props = new State(); props.setId("propsId"); props.setProp("prop1", "valueOfProp1"); props.setProp("prop2", "valueOfProp2"); DatasetState datasetState = new DatasetState(); datasetState.setDatasetUrn(this.datasetUrn); datasetState.incrementJobFailures(); this.sequence = new CommitSequence.Builder().withJobName("testjob").withDatasetUrn("testurn") .beginStep(FsRenameCommitStep.Builder.class).from(new Path("/ab/cd")).to(new Path("/ef/gh")).withProps(props) .endStep().beginStep(DatasetStateCommitStep.Builder.class).withDatasetUrn(this.datasetUrn) .withDatasetState(datasetState).withProps(props).endStep().build(); }
/** * This method does not actually move data, but it creates an {@link FsRenameCommitStep}. */ @Override protected void movePath(ParallelRunner parallelRunner, State state, Path src, Path dst, int branchId) throws IOException { log.info(String.format("Creating CommitStep for moving %s to %s", src, dst)); boolean overwrite = state.getPropAsBoolean(ConfigurationKeys.DATA_PUBLISHER_OVERWRITE_ENABLED, false); FsRenameCommitStep.Builder<?> builder = this.commitSequenceBuilder.get().beginStep(FsRenameCommitStep.Builder.class) .withProps(this.state).from(src).withSrcFs(this.writerFileSystemByBranches.get(branchId)).to(dst) .withDstFs(this.publisherFileSystemByBranches.get(branchId)); if (overwrite) { builder.overwrite(); } builder.endStep(); }
private synchronized void buildAndExecuteCommitSequence(CommitSequence.Builder builder, JobState.DatasetState datasetState, String datasetUrn) throws IOException { CommitSequence commitSequence = builder.addStep(buildDatasetStateCommitStep(datasetUrn, datasetState).get()).build(); this.jobContext.getCommitSequenceStore().get().put(commitSequence.getJobName(), datasetUrn, commitSequence); commitSequence.execute(); this.jobContext.getCommitSequenceStore().get().delete(commitSequence.getJobName(), datasetUrn); }
/** * Execute the {@link CommitStep}s in the order they are added to the commit sequence. */ public void execute() { try { for (CommitStep step : this.steps) { if (!step.isCompleted()) { step.execute(); } } } catch (Throwable t) { log.error("Commit failed for dataset " + this.datasetUrn, t); throw Throwables.propagate(t); } }
private void executeUnfinishedCommitSequences(String jobName) throws IOException { Preconditions.checkState(this.jobContext.getCommitSequenceStore().isPresent()); CommitSequenceStore commitSequenceStore = this.jobContext.getCommitSequenceStore().get(); for (String datasetUrn : commitSequenceStore.get(jobName)) { Optional<CommitSequence> commitSequence = commitSequenceStore.get(jobName, datasetUrn); if (commitSequence.isPresent()) { commitSequence.get().execute(); } commitSequenceStore.delete(jobName, datasetUrn); } }
@Override public boolean isCompleted() throws IOException { if (this.dstFs == null) { this.dstFs = getFileSystem(this.dstFsUri); } return this.dstFs.exists(this.dstPath); }
@Test public void testExecute() throws IOException { this.sequence.execute(); Assert.assertTrue(this.fs.exists(new Path(ROOT_DIR, "dir1/file2"))); Assert.assertTrue(this.fs.exists(new Path(ROOT_DIR, "dir2/file1"))); Assert.assertTrue(this.fs.exists(new Path(ROOT_DIR, "store/job-name/urn-job_id.jst"))); Assert.assertTrue(this.fs.exists(new Path(ROOT_DIR, "store/job-name/urn-current.jst"))); }
@Override public T withProps(State props) { return super.withProps(props); }
@Override public CommitStep build() throws IOException { Preconditions.checkNotNull(this.srcPath); Preconditions.checkNotNull(this.dstPath); return new FsRenameCommitStep(this); } }
public CommitSequence build() { Preconditions.checkState(!Strings.isNullOrEmpty(this.jobName), "Job name not specified for commit sequence"); Preconditions.checkState(!Strings.isNullOrEmpty(this.datasetUrn), "Dataset URN not specified for commit sequence"); Preconditions.checkState(!this.steps.isEmpty(), "No commit steps specified for the commit sequence"); return new CommitSequence(this); }
/** * Staging data cannot be cleaned if exactly once semantics is used, and the job has unfinished * commit sequences. */ private boolean canCleanStagingData(JobState jobState) throws IOException { return this.jobContext.getSemantics() != DeliverySemantics.EXACTLY_ONCE || !this.jobContext.getCommitSequenceStore() .get().exists(jobState.getJobName()); }
@Test public void testExecute() throws IOException { this.step.execute(); Assert.assertTrue(this.fs.exists(new Path(ROOT_DIR, "dir2/file"))); } }
private boolean isDataWriterForPartitionSafe(DataWriter dataWriter) { return dataWriter instanceof SpeculativeAttemptAwareConstruct && ((SpeculativeAttemptAwareConstruct) dataWriter).isSpeculativeAttemptSafe(); }
@Override public void execute() throws IOException { if (this.srcFs == null) { this.srcFs = getFileSystem(this.srcFsUri); } if (this.dstFs == null) { this.dstFs = getFileSystem(this.dstFsUri); } log.info(String.format("Moving %s to %s", this.srcPath, this.dstPath)); HadoopUtils.movePath(this.srcFs, this.srcPath, this.dstFs, this.dstPath, this.overwrite, this.dstFs.getConf()); } }
public boolean isSpeculativeExecutionSafe() { if (!this.writer.isPresent()) { return true; } if (!(this.writer.get() instanceof SpeculativeAttemptAwareConstruct)) { this.logger.info("Writer is not speculative safe: " + this.writer.get().getClass().toString()); return false; } return ((SpeculativeAttemptAwareConstruct) this.writer.get()).isSpeculativeAttemptSafe(); }
private FsRenameCommitStep(Builder<? extends Builder<?>> builder) throws IOException { super(builder); this.srcPath = builder.srcPath; this.dstPath = builder.dstPath; this.srcFs = builder.srcFs != null ? builder.srcFs : getFileSystem(this.props.getProp(ConfigurationKeys.FS_URI_KEY, ConfigurationKeys.LOCAL_FS_URI)); this.srcFsUri = this.srcFs.getUri().toString(); this.dstFs = builder.dstFs != null ? builder.dstFs : getFileSystem(this.props.getProp(ConfigurationKeys.FS_URI_KEY, ConfigurationKeys.LOCAL_FS_URI)); this.dstFsUri = this.dstFs.getUri().toString(); this.overwrite = builder.overwrite; }