T
- The data type that the comparator works on.@PublicEvolving public abstract class TypeComparator<T> extends Object implements Serializable
The methods in this interface depend not only on the record, but also on what fields of a record are
used for the comparison or hashing. That set of fields is typically a subset of a record's fields.
In general, this class assumes a contract on hash codes and equality the same way as defined for
Object.equals(Object)
Object.equals(Object)
Implementing classes are stateful, because several methods require to set one record as the reference for
comparisons and later comparing a candidate against it. Therefore, the classes implementing this interface are
not thread safe. The runtime will ensure that no instance is used twice in different threads, but will create
a copy for that purpose. It is hence imperative that the copies created by the duplicate()
method
share no state with the instance from which they were copied: they have to be deep copies.
Constructor and Description |
---|
TypeComparator() |
Modifier and Type | Method and Description |
---|---|
abstract int |
compare(T first,
T second)
Compares two records in object form.
|
int |
compareAgainstReference(Comparable[] keys) |
abstract int |
compareSerialized(DataInputView firstSource,
DataInputView secondSource)
Compares two records in serialized form.
|
abstract int |
compareToReference(TypeComparator<T> referencedComparator)
This method compares the element that has been set as reference in this type accessor, to the
element set as reference in the given type accessor.
|
abstract TypeComparator<T> |
duplicate()
Creates a copy of this class.
|
abstract boolean |
equalToReference(T candidate)
Checks, whether the given element is equal to the element that has been set as the comparison
reference in this comparator instance.
|
abstract int |
extractKeys(Object record,
Object[] target,
int index)
Extracts the key fields from a record.
|
abstract TypeComparator[] |
getFlatComparators()
Get the field comparators.
|
abstract int |
getNormalizeKeyLen()
Gets the number of bytes that the normalized key would maximally take.
|
abstract int |
hash(T record)
Computes a hash value for the given record.
|
abstract boolean |
invertNormalizedKey()
Flag whether normalized key comparisons should be inverted key should be interpreted
inverted, i.e.
|
abstract boolean |
isNormalizedKeyPrefixOnly(int keyBytes)
Checks, whether the given number of bytes for a normalized is only a prefix to determine the order of elements
of the data type for which this comparator provides the comparison methods.
|
abstract void |
putNormalizedKey(T record,
MemorySegment target,
int offset,
int numBytes)
Writes a normalized key for the given record into the target byte array, starting at the specified position
and writing exactly the given number of bytes.
|
abstract T |
readWithKeyDenormalization(T reuse,
DataInputView source)
Reads the record back while de-normalizing the key fields.
|
abstract void |
setReference(T toCompare)
Sets the given element as the comparison reference for future calls to
equalToReference(Object) and compareToReference(TypeComparator) . |
boolean |
supportsCompareAgainstReference() |
abstract boolean |
supportsNormalizedKey()
Checks whether the data type supports the creation of a normalized key for comparison.
|
abstract boolean |
supportsSerializationWithKeyNormalization()
Check whether this comparator supports to serialize the record in a format that replaces its keys by a normalized
key.
|
abstract void |
writeWithKeyNormalization(T record,
DataOutputView target)
Writes the record in such a fashion that all keys are normalizing and at the beginning of the serialized data.
|
public abstract int hash(T record)
The hash code is typically not used as it is in hash tables and for partitioning, but it is further scrambled to make sure that a projection of the hash values to a lower cardinality space is as results in a rather uniform value distribution. However, any collisions produced by this method cannot be undone. While it is NOT important to create hash codes that cover the full spectrum of bits in the integer, it IS important to avoid collisions when combining two value as much as possible.
record
- The record to be hashed.Object.hashCode()
public abstract void setReference(T toCompare)
equalToReference(Object)
and compareToReference(TypeComparator)
. This method
must set the given element into this comparator instance's state. If the comparison happens on a subset
of the fields from the record, this method may extract those fields.
A typical example for checking the equality of two elements is the following:
E e1 = ...;
E e2 = ...;
TypeComparator<E> acc = ...;
acc.setReference(e1);
boolean equal = acc.equalToReference(e2);
The rational behind this method is that elements are typically compared using certain features that
are extracted from them, (such de-serializing as a subset of fields). When setting the
reference, this extraction happens. The extraction needs happen only once per element,
even though an element is often compared to multiple other elements, such as when finding equal elements
in the process of grouping the elements.toCompare
- The element to set as the comparison reference.public abstract boolean equalToReference(T candidate)
candidate
- The candidate to check.setReference(Object)
public abstract int compareToReference(TypeComparator<T> referencedComparator)
e1
and e2
via a comparator, this method can be used the
following way.
E e1 = ...;
E e2 = ...;
TypeComparator<E> acc1 = ...;
TypeComparator<E> acc2 = ...;
acc1.setReference(e1);
acc2.setReference(e2);
int comp = acc1.compareToReference(acc2);
The rational behind this method is that elements are typically compared using certain features that
are extracted from them, (such de-serializing as a subset of fields). When setting the
reference, this extraction happens. The extraction needs happen only once per element,
even though an element is typically compared to many other elements when establishing a
sorted order. The actual comparison performed by this method may be very cheap, as it
happens on the extracted features.referencedComparator
- The type accessors where the element for comparison has been set
as reference.referencedAccessors
is smaller
than the reference value of this type accessor; a value greater than zero, if it is larger;
zero, if both are equal.setReference(Object)
public boolean supportsCompareAgainstReference()
public abstract int compare(T first, T second)
Comparator.compare(Object, Object)
.first
- The first record.second
- The second record.Comparator.compare(Object, Object)
.Comparator.compare(Object, Object)
public abstract int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException
Comparator.compare(Object, Object)
.
This method may de-serialize the records or compare them directly based on their binary representation.
firstSource
- The input view containing the first record.secondSource
- The input view containing the second record.Comparator.compare(Object, Object)
.IOException
- Thrown, if any of the input views raised an exception when reading the records.Comparator.compare(Object, Object)
public abstract boolean supportsNormalizedKey()
public abstract boolean supportsSerializationWithKeyNormalization()
public abstract int getNormalizeKeyLen()
Integer
.MAX_VALUE is interpreted as infinite.public abstract boolean isNormalizedKeyPrefixOnly(int keyBytes)
public abstract void putNormalizedKey(T record, MemorySegment target, int offset, int numBytes)
int byteI = bytes[i] & 0xFF;
If the meaningful part of the normalized key takes less than the given number of bytes, than it must be padded.
Padding is typically required for variable length data types, such as strings. The padding uses a special
character, either 0
or 0xff
, depending on whether shorter values are sorted to the beginning or
the end.
This method is similar to NormalizableKey.copyNormalizedKey(MemorySegment, int, int)
. In the case that
multiple fields of a record contribute to the normalized key, it is crucial that the fields align on the
byte field, i.e. that every field always takes up the exact same number of bytes.
record
- The record for which to create the normalized key.target
- The byte array into which to write the normalized key bytes.offset
- The offset in the byte array, where to start writing the normalized key bytes.numBytes
- The number of bytes to be written exactly.NormalizableKey.copyNormalizedKey(MemorySegment, int, int)
public abstract void writeWithKeyNormalization(T record, DataOutputView target) throws IOException
#supportsSerializationWithKeyNormalization()
allows to check that.record
- The record object into which to read the record data.target
- The stream to which to write the data,IOException
supportsSerializationWithKeyNormalization()
,
readWithKeyDenormalization(Object, DataInputView)
,
NormalizableKey.copyNormalizedKey(MemorySegment, int, int)
public abstract T readWithKeyDenormalization(T reuse, DataInputView source) throws IOException
#supportsSerializationWithKeyNormalization()
method.reuse
- The reuse object into which to read the record data.source
- The stream from which to read the data,IOException
supportsSerializationWithKeyNormalization()
,
writeWithKeyNormalization(Object, DataOutputView)
,
NormalizableKey.copyNormalizedKey(MemorySegment, int, int)
public abstract boolean invertNormalizedKey()
public abstract TypeComparator<T> duplicate()
public abstract int extractKeys(Object record, Object[] target, int index)
record
- The record that contains the key(s)target
- The array to write the key(s) into.index
- The offset of the target array to start writing into.public abstract TypeComparator[] getFlatComparators()
extractKeys(Object, Object[], int)
to provide interoperability between different record types. Note, that this should return at
least one Comparator and that the number of Comparators must match the number of extracted
keys.public int compareAgainstReference(Comparable[] keys)
Copyright © 2014–2020 The Apache Software Foundation. All rights reserved.