public class InPlaceMutableHashTable<T> extends AbstractMutableHashTable<T>
The memory is divided into three areas: - Bucket area: they contain bucket heads: an 8 byte pointer to the first link of a linked list in the record area - Record area: this contains the actual data in linked list elements. A linked list element starts with an 8 byte pointer to the next element, and then the record follows. - Staging area: This is a small, temporary storage area for writing updated records. This is needed, because before serializing a record, there is no way to know in advance how large will it be. Therefore, we can't serialize directly into the record area when we are doing an update, because if it turns out to be larger than the old record, then it would override some other record that happens to be after the old one in memory. The solution is to serialize to the staging area first, and then copy it to the place of the original if it has the same size, otherwise allocate a new linked list element at the end of the record area, and mark the old one as abandoned. This creates "holes" in the record area, so compactions are eventually needed.
Compaction happens by deleting everything in the bucket area, and then reinserting all elements. The reinsertion happens by forgetting the structure (the linked lists) of the record area, and reading it sequentially, and inserting all non-abandoned records, starting from the beginning of the record area. Note, that insertions never override a record that hasn't been read by the reinsertion sweep, because both the insertions and readings happen sequentially in the record area, and the insertions obviously never overtake the reading sweep.
Note: we have to abandon the old linked list element even when the updated record has a smaller size than the original, because otherwise we wouldn't know where the next record starts during a reinsertion sweep.
The number of buckets depends on how large are the records. The serializer might be able to tell us this, so in this case, we will calculate the number of buckets upfront, and won't do resizes. If the serializer doesn't know the size, then we start with a small number of buckets, and do resizes as more elements are inserted than the number of buckets.
The number of memory segments given to the staging area is usually one, because it just needs to hold one record.
Note: For hashing, we couldn't just take the lower bits, but have to use a proper hash function from MathUtils because of its avalanche property, so that changing only some high bits of the original value won't leave the lower bits of the hash unaffected. This is because when choosing the bucket for a record, we mask only the lower bits (see numBucketsMask). Lots of collisions would occur when, for example, the original value that is hashed is some bitset, where lots of different values that are different only in the higher bits will actually occur.
Modifier and Type | Class and Description |
---|---|
class |
InPlaceMutableHashTable.EntryIterator
WARNING: Doing any other operation on the table invalidates the iterator!
|
class |
InPlaceMutableHashTable.HashTableProber<PT>
A prober for accessing the table.
|
class |
InPlaceMutableHashTable.ReduceFacade
A facade for doing such operations on the hash table that are needed for a reduce operator
driver.
|
buildSideComparator, buildSideSerializer, closed, stateLock
Constructor and Description |
---|
InPlaceMutableHashTable(TypeSerializer<T> serializer,
TypeComparator<T> comparator,
List<MemorySegment> memory) |
Modifier and Type | Method and Description |
---|---|
void |
abort() |
void |
close()
Closes the hash table.
|
long |
getCapacity()
Gets the total capacity of this hash table, in bytes.
|
InPlaceMutableHashTable.EntryIterator |
getEntryIterator()
Returns an iterator that can be used to iterate over all the elements in the table.
|
List<MemorySegment> |
getFreeMemory() |
long |
getOccupancy()
Gets the number of bytes currently occupied in this hash table.
|
<PT> InPlaceMutableHashTable.HashTableProber<PT> |
getProber(TypeComparator<PT> probeTypeComparator,
TypePairComparator<PT,T> pairComparator) |
void |
insert(T record)
Inserts the given record into the hash table.
|
void |
insertOrReplaceRecord(T record)
Searches the hash table for a record with the given key.
|
void |
open()
Initialize the hash table
|
getBuildSideComparator, getBuildSideSerializer
public InPlaceMutableHashTable(TypeSerializer<T> serializer, TypeComparator<T> comparator, List<MemorySegment> memory)
public long getCapacity()
public long getOccupancy()
public void open()
open
in class AbstractMutableHashTable<T>
public void close()
AbstractMutableHashTable
close
in class AbstractMutableHashTable<T>
public void abort()
abort
in class AbstractMutableHashTable<T>
public List<MemorySegment> getFreeMemory()
getFreeMemory
in class AbstractMutableHashTable<T>
public void insertOrReplaceRecord(T record) throws IOException
insertOrReplaceRecord
in class AbstractMutableHashTable<T>
record
- The record to insert or to replace with.IOException
- (EOFException specifically, if memory ran out)public void insert(T record) throws IOException
insert
in class AbstractMutableHashTable<T>
record
- The record to insert.IOException
- (EOFException specifically, if memory ran out)public InPlaceMutableHashTable.EntryIterator getEntryIterator()
getEntryIterator
in class AbstractMutableHashTable<T>
public <PT> InPlaceMutableHashTable.HashTableProber<PT> getProber(TypeComparator<PT> probeTypeComparator, TypePairComparator<PT,T> pairComparator)
getProber
in class AbstractMutableHashTable<T>
Copyright © 2014–2024 The Apache Software Foundation. All rights reserved.