@InterfaceAudience.Public @InterfaceStability.Stable public abstract class CompositeRecordReader<K extends WritableComparable,V extends Writable,X extends Writable> extends Object implements Configurable
Modifier and Type | Field and Description |
---|---|
protected org.apache.hadoop.mapred.join.CompositeRecordReader.JoinCollector |
jc |
protected ComposableRecordReader<K,? extends V>[] |
kids |
Constructor and Description |
---|
CompositeRecordReader(int id,
int capacity,
Class<? extends WritableComparator> cmpcl)
Create a RecordReader with capacity children to position
id in the parent reader.
|
Modifier and Type | Method and Description |
---|---|
void |
accept(org.apache.hadoop.mapred.join.CompositeRecordReader.JoinCollector jc,
K key)
If key provided matches that of this Composite, give JoinCollector
iterator over values it may emit.
|
void |
add(ComposableRecordReader<K,? extends V> rr)
Add a RecordReader to this collection.
|
void |
close()
Close all child RRs.
|
protected abstract boolean |
combine(Object[] srcs,
TupleWritable value) |
int |
compareTo(ComposableRecordReader<K,?> other)
Implement Comparable contract (compare key of join or head of heap
with that of another).
|
protected TupleWritable |
createInternalValue()
Create a value to be used internally for joins.
|
K |
createKey()
Create a new key value common to all child RRs.
|
protected void |
fillJoinCollector(K iterkey)
For all child RRs offering the key provided, obtain an iterator
at that position in the JoinCollector.
|
protected WritableComparator |
getComparator()
Return comparator defining the ordering for RecordReaders in this
composite.
|
Configuration |
getConf()
Return the configuration used by this object.
|
protected abstract ResetableIterator<X> |
getDelegate()
Obtain an iterator over the child RRs apropos of the value type
ultimately emitted from this join.
|
long |
getPos()
Unsupported (returns zero in all cases).
|
float |
getProgress()
Report progress as the minimum of all child RR progress.
|
protected PriorityQueue<ComposableRecordReader<K,?>> |
getRecordReaderQueue()
Return sorted list of RecordReaders for this composite.
|
boolean |
hasNext()
Return true if it is possible that this could emit more values.
|
int |
id()
Return the position in the collector this class occupies.
|
K |
key()
Return the key for the current join or the value at the top of the
RecordReader heap.
|
void |
key(K key)
Clone the key at the top of this RR into the given object.
|
void |
setConf(Configuration conf)
Set the configuration to be used by this object.
|
void |
skip(K key)
Pass skip key to child RRs.
|
protected final org.apache.hadoop.mapred.join.CompositeRecordReader.JoinCollector jc
protected final ComposableRecordReader<K extends WritableComparable,? extends V extends Writable>[] kids
public CompositeRecordReader(int id, int capacity, Class<? extends WritableComparator> cmpcl) throws IOException
IOException
protected abstract boolean combine(Object[] srcs, TupleWritable value)
public int id()
public void setConf(Configuration conf)
setConf
in interface Configurable
public Configuration getConf()
getConf
in interface Configurable
protected PriorityQueue<ComposableRecordReader<K,?>> getRecordReaderQueue()
protected WritableComparator getComparator()
public void add(ComposableRecordReader<K,? extends V> rr) throws IOException
IOException
public K key()
public void key(K key) throws IOException
IOException
public boolean hasNext()
public void skip(K key) throws IOException
IOException
protected abstract ResetableIterator<X> getDelegate()
public void accept(org.apache.hadoop.mapred.join.CompositeRecordReader.JoinCollector jc, K key) throws IOException
IOException
protected void fillJoinCollector(K iterkey) throws IOException
IOException
public int compareTo(ComposableRecordReader<K,?> other)
public K createKey()
ClassCastException
- if key classes differ.protected TupleWritable createInternalValue()
public long getPos() throws IOException
IOException
public void close() throws IOException
IOException
public float getProgress() throws IOException
IOException
Copyright © 2017 Apache Software Foundation. All rights reserved.