Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Create workunitstream for CompactionSource #1826

Closed
wants to merge 1 commit into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -18,32 +18,39 @@
import gobblin.runtime.JobState;
import gobblin.runtime.task.TaskUtils;
import gobblin.source.Source;
import gobblin.source.WorkUnitStreamSource;
import gobblin.source.extractor.Extractor;
import gobblin.source.workunit.BasicWorkUnitStream;
import gobblin.source.workunit.WorkUnit;
import gobblin.source.workunit.WorkUnitStream;
import gobblin.util.HadoopUtils;
import lombok.extern.slf4j.Slf4j;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;

import java.io.IOException;
import java.net.URI;
import java.util.List;
import java.util.Iterator;
import java.util.NoSuchElementException;
import java.util.UUID;
import java.util.concurrent.*;
import java.util.List;

/**
* A compaction source derived from {@link Source} which uses {@link DefaultFileSystemGlobFinder} to find all
* {@link Dataset}s. Use {@link CompactionSuite#getDatasetsFinderVerifiers()} to guarantee a given dataset has passed
* all verification. Each found dataset will be serialized to {@link WorkUnit} by {@link CompactionSuite#save(Dataset, State)}
*/
@Slf4j
public class CompactionSource implements Source<String, String> {
public class CompactionSource implements WorkUnitStreamSource<String, String> {
private CompactionSuite suite;
private Path tmpJobDir;
private FileSystem fs;
private ExecutorService service;

@Override
public List<WorkUnit> getWorkunits(SourceState state) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You should remove getWorkUnits(SourceState), it is better to only have to maintain one functionally equivalent method.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed

List<WorkUnit> workUnits = Lists.newArrayList();

Expand Down Expand Up @@ -84,6 +91,135 @@ public List<WorkUnit> getWorkunits(SourceState state) {
return workUnits;
}

@Override
public WorkUnitStream getWorkunitStream(SourceState state) {
service = Executors.newSingleThreadExecutor();

try {
fs = getSourceFileSystem(state);
suite = CompactionSuiteUtils.getCompactionSuiteFactory(state).createSuite(state);

initJobDir(state);
copyJarDependencies(state);
List<CompactionVerifier> verifiers = suite.getDatasetsFinderVerifiers();
DatasetsFinder finder = DatasetUtils.instantiateDatasetFinder(state.getProperties(),
getSourceFileSystem(state),
DefaultFileSystemGlobFinder.class.getName());

List<Dataset> datasets = finder.findDatasets();
CompactionDatasetIterator iterator = new CompactionDatasetIterator(datasets, verifiers);
service.submit(iterator.getDatasetProcessor());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You probably want multiple data processors right? Note that will also require to transform the List into something like a Spliterator (although that is only available on java 8)

return new BasicWorkUnitStream.Builder (iterator).build();
} catch (IOException e) {
throw new RuntimeException(e);
}
}


/**
* Iterator that provides {@link WorkUnit}s for all verified {@link Dataset}s
*/
public class CompactionDatasetIterator implements Iterator<WorkUnit> {
private LinkedBlockingDeque<WorkUnit> workUnits;

private List<Dataset> datasets;
private List<CompactionVerifier> verifiers;
private WorkUnit last;
private volatile int unProcessed;
private volatile int success;

/**
* Constructor
*/
public CompactionDatasetIterator (List<Dataset> datasets, List<CompactionVerifier> verifiers) {
this.datasets = datasets;
this.workUnits = new LinkedBlockingDeque<>();
this.verifiers = verifiers;
this.unProcessed = datasets.size();
this.success = 0;
this.last = null;
}

/**
* Check if any {@link WorkUnit} is available. The producer is {@link CompactionDatasetIterator#getDatasetProcessor()}
* @return true when a new {@link WorkUnit} is available
* false when producer exits
*/
public boolean hasNext () {
try {
while (true) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Would this work?

while(true) {
   if (last != null) return true;
   if (this.unProcessed <= 0 && this.workUnits.isEmpty()) return false;
   this.last = this.workUnits.poll(1, TimeUnit.SECONDS);
}

if (last != null) {
log.debug ("hasNext() is true because cache is not empty");
return true;
}
if (this.unProcessed > 0) {
last = this.workUnits.poll(1, TimeUnit.SECONDS);
if (last == null)
log.debug ("Waiting for producer to complete...");
} else {
if (workUnits.size() > 0) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it seems you should add last = this.workUnits.poll() here?

log.debug ("hasNext() has new element");
return true;
}
log.debug ("hasNext() returns false because producer is complete");
return false;
}
}
} catch (InterruptedException e) {
log.error(e.toString());
return false;
}
}

/**
* Obtain next available {@link WorkUnit}.
* Block the consumer thread until a new {@link WorkUnit} is provided. Otherwise throw an exception
*/
public WorkUnit next () {
if (hasNext()) {
if (last != null) {
WorkUnit tmp = last;
last = null;
log.debug ("next() pops out a workunit");
return tmp;
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

you should probably have else { throw new IllegalStateException()} to catch the case where hasNext() returns true but last doesn't get populated.

}

throw new NoSuchElementException ("work units queue has been exhausted");
}

public void remove() {
throw new UnsupportedOperationException("No remove supported on " + this.getClass().getName());
}

public Callable<Integer> getDatasetProcessor () {
return new Callable<Integer>() {
@Override
public Integer call() throws Exception {
for (Dataset dataset: datasets) {
// all verifier should be passed before we compact the dataset
boolean verificationPassed = true;
if (verifiers != null) {
for (CompactionVerifier verifier : verifiers) {
if (!verifier.verify(dataset)) {
verificationPassed = false;
break;
}
}
}
if (verificationPassed) {
success++;
workUnits.add (createWorkUnit(dataset));
}
unProcessed--;
}

return success;
}
};
}
}

protected WorkUnit createWorkUnit (Dataset dataset) throws IOException {
WorkUnit workUnit = new WorkUnit();
TaskUtils.setTaskFactoryClass(workUnit, MRCompactionTaskFactory.class);
Expand All @@ -101,6 +237,7 @@ public void shutdown (SourceState state) {
try {
boolean f = fs.delete(this.tmpJobDir, true);
log.info("Job dir is removed from {} with status {}", this.tmpJobDir, f);
service.shutdown();
} catch (IOException e) {
throw new RuntimeException(e);
}
Expand Down Expand Up @@ -159,4 +296,5 @@ private void copyJarDependencies (State state) throws IOException {
}
}
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -155,4 +155,26 @@ private EmbeddedGobblin createEmbeddedGobblin (String name, String basePath) {
.setConfiguration(TimeBasedSubDirDatasetsFinder.COMPACTION_TIMEBASED_MIN_TIME_AGO, "1d");

}

@Test
public void testWorkUnitStream () throws Exception {
File basePath = Files.createTempDir();
basePath.deleteOnExit();

File jobDir1 = new File(basePath, "Identity/MemberAccount/minutely/2017/04/03/10/20_30/run_2017-04-03-10-20");
File jobDir2 = new File(basePath, "Identity/MemberAccount/minutely/2017/04/03/11/20_30/run_2017-04-03-10-21");
File jobDir3 = new File(basePath, "Identity/MemberAccount/minutely/2017/04/03/12/20_30/run_2017-04-03-10-22");
Assert.assertTrue(jobDir1.mkdirs());
Assert.assertTrue(jobDir2.mkdirs());
Assert.assertTrue(jobDir3.mkdirs());

GenericRecord r1 = createRandomRecord();
writeFileWithContent(jobDir1, "file1", r1, 20);
writeFileWithContent(jobDir2, "file1", r1, 20);
writeFileWithContent(jobDir3, "file1", r1, 20);

EmbeddedGobblin embeddedGobblin = createEmbeddedGobblin("workunit_stream", basePath.getAbsolutePath().toString());
JobExecutionResult result = embeddedGobblin.run();
Assert.assertTrue(result.isSuccessful());
}
}