-
Notifications
You must be signed in to change notification settings - Fork 262
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
KITE-1025 Use CombineFileInputFormat
Wrap Avro and Parquet inputs with CombineFileInputFormat so that multiple small files don't result in multiple input splits and MapReduce tasks.
- Loading branch information
Showing
5 changed files
with
277 additions
and
22 deletions.
There are no files selected for viewing
86 changes: 86 additions & 0 deletions
86
...a-core/src/main/java/org/kitesdk/data/spi/filesystem/AbstractCombineFileRecordReader.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,86 @@ | ||
/* | ||
* Licensed under the Apache License, Version 2.0 (the "License"); | ||
* you may not use this file except in compliance with the License. | ||
* You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package org.kitesdk.data.spi.filesystem; | ||
|
||
import java.io.IOException; | ||
import org.apache.hadoop.mapreduce.InputSplit; | ||
import org.apache.hadoop.mapreduce.RecordReader; | ||
import org.apache.hadoop.mapreduce.TaskAttemptContext; | ||
import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit; | ||
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; | ||
import org.apache.hadoop.mapreduce.lib.input.FileSplit; | ||
|
||
/** | ||
* Base class for wrapping file-based record readers with CombineFileInputFormat functionality. This allows multiple | ||
* files to be combined into a single InputSplit, with the main pre-requisite being that the InputFormat being wrapped | ||
* must make use of FileInputSplits. | ||
* <p/> | ||
* Sub-classes need only implement the method to instantiate an instance of their owning InputFormat class. | ||
*/ | ||
abstract class AbstractCombineFileRecordReader<K, V> extends RecordReader<K, V> { | ||
|
||
private int idx; | ||
private RecordReader<K, V> delegate; | ||
|
||
public AbstractCombineFileRecordReader(CombineFileSplit split, TaskAttemptContext context, Integer idx) { | ||
this.idx = idx; | ||
} | ||
|
||
/** | ||
* Single extension point. Returns an instance of the underlying InputFormat being used. | ||
* | ||
* @return an instance of the underlying FileInputFormat | ||
*/ | ||
abstract FileInputFormat<K, V> getInputFormat(); | ||
|
||
@Override | ||
public void initialize(InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException { | ||
if (delegate != null) { | ||
delegate.close(); | ||
} | ||
CombineFileSplit combineSplit = (CombineFileSplit) split; | ||
FileSplit fileSplit = new FileSplit(combineSplit.getPath(idx), combineSplit.getOffset(idx), | ||
combineSplit.getLength(idx), combineSplit.getLocations()); | ||
delegate = getInputFormat().createRecordReader(fileSplit, context); | ||
delegate.initialize(fileSplit, context); | ||
} | ||
|
||
@Override | ||
public boolean nextKeyValue() throws IOException, InterruptedException { | ||
return delegate.nextKeyValue(); | ||
} | ||
|
||
@Override | ||
public K getCurrentKey() throws IOException, InterruptedException { | ||
return delegate.getCurrentKey(); | ||
} | ||
|
||
@Override | ||
public V getCurrentValue() throws IOException, InterruptedException { | ||
return delegate.getCurrentValue(); | ||
} | ||
|
||
@Override | ||
public float getProgress() throws IOException, InterruptedException { | ||
return delegate.getProgress(); | ||
} | ||
|
||
@Override | ||
public void close() throws IOException { | ||
if (delegate != null) { | ||
delegate.close(); | ||
delegate = null; | ||
} | ||
} | ||
} |
111 changes: 111 additions & 0 deletions
111
...ore/src/main/java/org/kitesdk/data/spi/filesystem/AbstractKiteCombineFileInputFormat.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,111 @@ | ||
/* | ||
* Licensed under the Apache License, Version 2.0 (the "License"); | ||
* you may not use this file except in compliance with the License. | ||
* You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package org.kitesdk.data.spi.filesystem; | ||
|
||
import com.google.common.collect.Lists; | ||
import java.io.DataInput; | ||
import java.io.DataOutput; | ||
import java.io.IOException; | ||
import java.util.List; | ||
import org.apache.hadoop.io.Writable; | ||
import org.apache.hadoop.mapreduce.InputSplit; | ||
import org.apache.hadoop.mapreduce.JobContext; | ||
import org.apache.hadoop.mapreduce.RecordReader; | ||
import org.apache.hadoop.mapreduce.TaskAttemptContext; | ||
import org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat; | ||
import org.apache.hadoop.mapreduce.lib.input.CombineFileRecordReader; | ||
import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit; | ||
|
||
/** | ||
* A Kite-specific subclass of CombineFileInputFormat and CombineFileRecordReader to work around the fact that | ||
* Crunch does special handling of CombineFileSplits internally. | ||
* <p/> | ||
* The classes in this file don't add any additional functionality to CombineFileInputFormat or CombineFileRecordReader, | ||
* they just ensure that KiteCombineFileSplits are being used (instead of CombineFileSplit) to avoid CrunchInputFormat | ||
* from interpreting them as coming from Crunch. | ||
*/ | ||
abstract class AbstractKiteCombineFileInputFormat<K, V> extends CombineFileInputFormat<K, V> { | ||
|
||
/** | ||
* A wrapper around CombineFileSplit which simply allows us to avoid using CombineFileSplit directly, as Crunch | ||
* has special handling for CombineFileSplit. | ||
*/ | ||
static class KiteCombineFileSplit extends InputSplit implements Writable { | ||
|
||
private CombineFileSplit delegate; | ||
|
||
public KiteCombineFileSplit() { | ||
// Needed for Writable interface | ||
} | ||
|
||
public KiteCombineFileSplit(CombineFileSplit delegate) { | ||
this.delegate = delegate; | ||
} | ||
|
||
|
||
public void readFields(DataInput in) throws IOException { | ||
delegate = new CombineFileSplit(); | ||
delegate.readFields(in); | ||
} | ||
|
||
public void write(DataOutput out) throws IOException { | ||
delegate.write(out); | ||
} | ||
|
||
@Override | ||
public long getLength() throws IOException, InterruptedException { | ||
return delegate.getLength(); | ||
} | ||
|
||
@Override | ||
public String[] getLocations() throws IOException, InterruptedException { | ||
return delegate.getLocations(); | ||
} | ||
|
||
CombineFileSplit getCombineFileSplit() { | ||
return delegate; | ||
} | ||
} | ||
|
||
abstract Class<? extends AbstractCombineFileRecordReader> getRecordReaderClass(); | ||
|
||
private static class KiteCombineFileRecordReader extends CombineFileRecordReader { | ||
|
||
@SuppressWarnings("unchecked") | ||
public KiteCombineFileRecordReader(KiteCombineFileSplit kiteCombineSplit, TaskAttemptContext context, Class rrClass) throws IOException { | ||
super(kiteCombineSplit.getCombineFileSplit(), context, rrClass); | ||
} | ||
|
||
@Override | ||
public void initialize(InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException { | ||
KiteCombineFileSplit kiteCombineFileSplit = (KiteCombineFileSplit) split; | ||
super.initialize(kiteCombineFileSplit.getCombineFileSplit(), context); | ||
} | ||
} | ||
|
||
@SuppressWarnings("unchecked") | ||
@Override | ||
public RecordReader<K, V> createRecordReader(InputSplit inputSplit, TaskAttemptContext taskContext) throws IOException { | ||
return new KiteCombineFileRecordReader((KiteCombineFileSplit) inputSplit, taskContext, getRecordReaderClass()); | ||
} | ||
|
||
@Override | ||
public List<InputSplit> getSplits(JobContext job) throws IOException { | ||
List<InputSplit> kiteCombineSplits = Lists.newArrayList(); | ||
for (InputSplit inputSplit : super.getSplits(job)) { | ||
kiteCombineSplits.add(new KiteCombineFileSplit((CombineFileSplit) inputSplit)); | ||
} | ||
return kiteCombineSplits; | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters