-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Best effort to find locations for input splits #2223
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,14 +20,25 @@ | |
| package io.druid.indexer.hadoop; | ||
|
|
||
| import com.fasterxml.jackson.core.type.TypeReference; | ||
| import com.google.common.annotations.VisibleForTesting; | ||
| import com.google.common.base.Preconditions; | ||
| import com.google.common.base.Supplier; | ||
| import com.google.common.collect.Iterables; | ||
| import com.google.common.collect.Lists; | ||
| import com.google.common.collect.Sets; | ||
| import com.metamx.common.ISE; | ||
| import com.metamx.common.Pair; | ||
| import com.metamx.common.logger.Logger; | ||
| import io.druid.collections.CountingMap; | ||
| import io.druid.data.input.InputRow; | ||
| import io.druid.indexer.HadoopDruidIndexerConfig; | ||
| import io.druid.indexer.JobHelper; | ||
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.hadoop.fs.Path; | ||
| import org.apache.hadoop.io.NullWritable; | ||
| import org.apache.hadoop.mapred.FileInputFormat; | ||
| import org.apache.hadoop.mapred.JobConf; | ||
| import org.apache.hadoop.mapred.TextInputFormat; | ||
| import org.apache.hadoop.mapreduce.InputFormat; | ||
| import org.apache.hadoop.mapreduce.InputSplit; | ||
| import org.apache.hadoop.mapreduce.JobContext; | ||
|
|
@@ -36,9 +47,13 @@ | |
|
|
||
| import java.io.IOException; | ||
| import java.util.ArrayList; | ||
| import java.util.Arrays; | ||
| import java.util.Collections; | ||
| import java.util.Comparator; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.TreeSet; | ||
| import java.util.concurrent.atomic.AtomicLong; | ||
|
|
||
| public class DatasourceInputFormat extends InputFormat<NullWritable, InputRow> | ||
| { | ||
|
|
@@ -89,9 +104,11 @@ public int compare(WindowedDataSegment s1, WindowedDataSegment s2) | |
| List<WindowedDataSegment> list = new ArrayList<>(); | ||
| long size = 0; | ||
|
|
||
| JobConf dummyConf = new JobConf(); | ||
| org.apache.hadoop.mapred.InputFormat fio = supplier.get(); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. both these lines can be moved inside toDataSourceSplit() itself.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. JobConf is rather a big object and I wanted to reuse it if possible.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. ok, dint realize it was used twice.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
How come is it a big object? You literally create it completely empty. |
||
| for (WindowedDataSegment segment : segments) { | ||
| if (size + segment.getSegment().getSize() > maxSize && size > 0) { | ||
| splits.add(new DatasourceInputSplit(list)); | ||
| splits.add(toDataSourceSplit(list, fio, dummyConf)); | ||
| list = Lists.newArrayList(); | ||
| size = 0; | ||
| } | ||
|
|
@@ -101,7 +118,7 @@ public int compare(WindowedDataSegment s1, WindowedDataSegment s2) | |
| } | ||
|
|
||
| if (list.size() > 0) { | ||
| splits.add(new DatasourceInputSplit(list)); | ||
| splits.add(toDataSourceSplit(list, fio, dummyConf)); | ||
| } | ||
|
|
||
| logger.info("Number of splits [%d]", splits.size()); | ||
|
|
@@ -116,4 +133,85 @@ public RecordReader<NullWritable, InputRow> createRecordReader( | |
| { | ||
| return new DatasourceRecordReader(); | ||
| } | ||
|
|
||
| private Supplier<org.apache.hadoop.mapred.InputFormat> supplier = new Supplier<org.apache.hadoop.mapred.InputFormat>() | ||
| { | ||
| @Override | ||
| public org.apache.hadoop.mapred.InputFormat get() | ||
| { | ||
| return new TextInputFormat(); | ||
| } | ||
| }; | ||
|
|
||
| @VisibleForTesting | ||
| DatasourceInputFormat setSupplier(Supplier<org.apache.hadoop.mapred.InputFormat> supplier) { | ||
| this.supplier = supplier; | ||
| return this; | ||
| } | ||
|
|
||
| private DatasourceInputSplit toDataSourceSplit( | ||
| List<WindowedDataSegment> segments, | ||
| org.apache.hadoop.mapred.InputFormat fio, | ||
| JobConf conf | ||
| ) | ||
| { | ||
| String[] locations = null; | ||
| try { | ||
| locations = getFrequentLocations(segments, fio, conf); | ||
| } | ||
| catch (Exception e) { | ||
| logger.error("Exception thrown finding location of splits", e); | ||
| } | ||
| return new DatasourceInputSplit(segments, locations); | ||
| } | ||
|
|
||
| private String[] getFrequentLocations( | ||
| List<WindowedDataSegment> segments, | ||
| org.apache.hadoop.mapred.InputFormat fio, | ||
| JobConf conf | ||
| ) throws IOException | ||
| { | ||
| Iterable<String> locations = Collections.emptyList(); | ||
| for (WindowedDataSegment segment : segments) { | ||
| FileInputFormat.setInputPaths(conf, new Path(JobHelper.getURIFromSegment(segment.getSegment()))); | ||
| for (org.apache.hadoop.mapred.InputSplit split : fio.getSplits(conf, 1)) { | ||
| locations = Iterables.concat(locations, Arrays.asList(split.getLocations())); | ||
| } | ||
| } | ||
| return getFrequentLocations(locations); | ||
| } | ||
|
|
||
| private static String[] getFrequentLocations(Iterable<String> hosts) { | ||
|
|
||
| final CountingMap<String> counter = new CountingMap<>(); | ||
| for (String location : hosts) { | ||
| counter.add(location, 1); | ||
| } | ||
|
|
||
| final TreeSet<Pair<Long, String>> sorted = Sets.<Pair<Long, String>>newTreeSet( | ||
| new Comparator<Pair<Long, String>>() | ||
| { | ||
| @Override | ||
| public int compare(Pair<Long, String> o1, Pair<Long, String> o2) | ||
| { | ||
| int compare = o2.lhs.compareTo(o1.lhs); // descending | ||
| if (compare == 0) { | ||
| compare = o1.rhs.compareTo(o2.rhs); // ascending | ||
| } | ||
| return compare; | ||
| } | ||
| } | ||
| ); | ||
|
|
||
| for (Map.Entry<String, AtomicLong> entry : counter.entrySet()) { | ||
| sorted.add(Pair.of(entry.getValue().get(), entry.getKey())); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: using TreeMap instead of TreeSet would avoid Pair creation. |
||
| } | ||
|
|
||
| // use default replication factor, if possible | ||
| final List<String> locations = Lists.newArrayListWithCapacity(3); | ||
| for (Pair<Long, String> frequent : Iterables.limit(sorted, 3)) { | ||
| locations.add(frequent.rhs); | ||
| } | ||
| return locations.toArray(new String[locations.size()]); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -33,17 +33,21 @@ | |
|
|
||
| public class DatasourceInputSplit extends InputSplit implements Writable | ||
| { | ||
| private static final String[] EMPTY_STR_ARRAY = new String[0]; | ||
|
|
||
| private List<WindowedDataSegment> segments = null; | ||
| private String[] locations = null; | ||
|
|
||
| //required for deserialization | ||
| public DatasourceInputSplit() | ||
| { | ||
| } | ||
|
|
||
| public DatasourceInputSplit(@NotNull List<WindowedDataSegment> segments) | ||
| public DatasourceInputSplit(@NotNull List<WindowedDataSegment> segments, String[] locations) | ||
| { | ||
| Preconditions.checkArgument(segments != null && segments.size() > 0, "no segments"); | ||
| this.segments = segments; | ||
| this.locations = locations == null ? EMPTY_STR_ARRAY : locations; | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -59,7 +63,7 @@ public long getLength() throws IOException, InterruptedException | |
| @Override | ||
| public String[] getLocations() throws IOException, InterruptedException | ||
| { | ||
| return new String[]{}; | ||
| return locations; | ||
| } | ||
|
|
||
| public List<WindowedDataSegment> getSegments() | ||
|
|
@@ -71,6 +75,10 @@ public List<WindowedDataSegment> getSegments() | |
| public void write(DataOutput out) throws IOException | ||
| { | ||
| out.writeUTF(HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString(segments)); | ||
| out.writeInt(locations.length); | ||
| for (String location : locations) { | ||
| out.writeUTF(location); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -82,5 +90,9 @@ public void readFields(DataInput in) throws IOException | |
| { | ||
| } | ||
| ); | ||
| locations = new String[in.readInt()]; | ||
| for (int i = 0; i < locations.length; i++) { | ||
| locations[i] = in.readUTF(); | ||
| } | ||
| } | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. can you add/update serde tests in DatasourceInputSplitTest?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. done |
||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
How is this supposed to work? The first line of this method says:
However, the values in this configuration are ignored, and a blank "dummy" config is passed on instead. This causes exceptions like described in #5135
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think, you're right . rewriting it to use
org.apache.hadoop.mapreduce.lib.input.FileInputFormatand supplied JobContext in getSplits(JobContext) arg would probably solve #5135 . Please feel free to submit a PR with patch that does so after its verification.