BT
- The type of the build side records.PT
- The type of the probe side records.public class HashPartition<BT,PT> extends AbstractPagedInputView implements SeekableDataInputView
Modifier and Type | Class and Description |
---|---|
protected static class |
HashPartition.BuildSideBuffer |
Modifier and Type | Field and Description |
---|---|
protected boolean |
furtherPartitioning |
protected int |
nextOverflowBucket |
protected int |
numOverflowSegments |
protected MemorySegment[] |
overflowSegments |
protected MemorySegment[] |
partitionBuffers |
protected ChannelWriterOutputView |
probeSideBuffer |
protected BlockChannelWriter<MemorySegment> |
probeSideChannel |
protected long |
probeSideRecordCounter |
protected int |
recursionLevel |
headerLength
Modifier and Type | Method and Description |
---|---|
void |
clearAllMemory(List<MemorySegment> target) |
void |
finalizeBuildPhase(IOManager ioAccess,
FileIOChannel.Enumerator probeChannelEnumerator,
LinkedBlockingQueue<MemorySegment> bufferReturnQueue) |
int |
finalizeProbePhase(List<MemorySegment> freeMemory,
List<HashPartition<BT,PT>> spilledPartitions,
boolean keepUnprobedSpilledPartitions) |
int |
getBuildSideBlockCount() |
BlockChannelWriter<MemorySegment> |
getBuildSideChannel() |
long |
getBuildSideRecordCount() |
protected int |
getLimitForSegment(MemorySegment segment)
Gets the limit for reading bytes from the given memory segment.
|
int |
getNumOccupiedMemorySegments()
Gets the number of memory segments used by this partition, which includes build side memory
buffers and overflow memory segments.
|
int |
getPartitionNumber()
Gets the partition number of this partition.
|
int |
getProbeSideBlockCount() |
BlockChannelWriter<MemorySegment> |
getProbeSideChannel() |
long |
getProbeSideRecordCount() |
int |
getRecursionLevel()
Gets this partition's recursion level.
|
long |
insertIntoBuildBuffer(BT record)
Inserts the given object into the current buffer.
|
void |
insertIntoProbeBuffer(PT record)
Inserts the given record into the probe side buffers.
|
boolean |
isInMemory()
Checks whether this partition is in memory or spilled.
|
protected MemorySegment |
nextSegment(MemorySegment current)
The method by which concrete subclasses realize page crossing.
|
void |
prepareProbePhase(IOManager ioAccess,
FileIOChannel.Enumerator probeChannelEnumerator,
LinkedBlockingQueue<MemorySegment> bufferReturnQueue) |
protected void |
setFurtherPatitioning(boolean v) |
void |
setReadPosition(long pointer)
Sets the read pointer to the given position.
|
int |
spillPartition(List<MemorySegment> target,
IOManager ioAccess,
FileIOChannel.ID targetChannel,
LinkedBlockingQueue<MemorySegment> bufferReturnQueue)
Spills this partition to disk and sets it up such that it continues spilling records that are
added to it.
|
advance, clear, doAdvance, getCurrentPositionInSegment, getCurrentSegment, getCurrentSegmentLimit, getHeaderLength, read, read, readBoolean, readByte, readChar, readDouble, readFloat, readFully, readFully, readInt, readLine, readLong, readShort, readUnsignedByte, readUnsignedShort, readUTF, seekInput, skipBytes, skipBytesToRead
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
read, read, skipBytesToRead
readBoolean, readByte, readChar, readDouble, readFloat, readFully, readFully, readInt, readLine, readLong, readShort, readUnsignedByte, readUnsignedShort, readUTF, skipBytes
protected MemorySegment[] overflowSegments
protected int numOverflowSegments
protected int nextOverflowBucket
protected MemorySegment[] partitionBuffers
protected ChannelWriterOutputView probeSideBuffer
protected long probeSideRecordCounter
protected int recursionLevel
protected BlockChannelWriter<MemorySegment> probeSideChannel
protected boolean furtherPartitioning
protected void setFurtherPatitioning(boolean v)
public int getPartitionNumber()
public int getRecursionLevel()
public final boolean isInMemory()
public int getNumOccupiedMemorySegments()
public int getBuildSideBlockCount()
public int getProbeSideBlockCount()
public long getBuildSideRecordCount()
public long getProbeSideRecordCount()
public BlockChannelWriter<MemorySegment> getBuildSideChannel()
public BlockChannelWriter<MemorySegment> getProbeSideChannel()
public final long insertIntoBuildBuffer(BT record) throws IOException
record
- The object to be written to the partition.-1
, if the partition is
spilled.IOException
- Thrown, when this is a spilled partition and the write failed.public final void insertIntoProbeBuffer(PT record) throws IOException
If this method is invoked when the partition is still being built, it has undefined behavior.
record
- The record to be inserted into the probe side buffers.IOException
- Thrown, if the buffer is full, needs to be spilled, and spilling causes
an error.public int spillPartition(List<MemorySegment> target, IOManager ioAccess, FileIOChannel.ID targetChannel, LinkedBlockingQueue<MemorySegment> bufferReturnQueue) throws IOException
target
- The list to which memory segments from overflow buckets are added.ioAccess
- The I/O manager to be used to create a writer to disk.targetChannel
- The id of the target channel for this partition.IOException
- Thrown, if the writing failed.public void finalizeBuildPhase(IOManager ioAccess, FileIOChannel.Enumerator probeChannelEnumerator, LinkedBlockingQueue<MemorySegment> bufferReturnQueue) throws IOException
IOException
public int finalizeProbePhase(List<MemorySegment> freeMemory, List<HashPartition<BT,PT>> spilledPartitions, boolean keepUnprobedSpilledPartitions) throws IOException
keepUnprobedSpilledPartitions
- If true then partitions that were spilled but received
no further probe requests will be retained; used for build-side outer joins.IOException
public void clearAllMemory(List<MemorySegment> target)
public void prepareProbePhase(IOManager ioAccess, FileIOChannel.Enumerator probeChannelEnumerator, LinkedBlockingQueue<MemorySegment> bufferReturnQueue) throws IOException
IOException
public void setReadPosition(long pointer)
SeekableDataInputView
setReadPosition
in interface SeekableDataInputView
pointer
- The new read position.protected MemorySegment nextSegment(MemorySegment current) throws IOException
AbstractPagedInputView
EOFException
.nextSegment
in class AbstractPagedInputView
current
- The current page that was read to its limit. May be null
, if this
method is invoked for the first time.null
. If the
input is exhausted, an EOFException
must be thrown instead.EOFException
- Thrown, if no further segment is available.IOException
- Thrown, if the method cannot provide the next page due to an I/O related
problem.protected int getLimitForSegment(MemorySegment segment)
AbstractPagedInputView
getLimitForSegment
in class AbstractPagedInputView
segment
- The segment to determine the limit for.Copyright © 2014–2024 The Apache Software Foundation. All rights reserved.