if (row.size() <= 1) {
if (row.isEmpty()) { // Maybe the row got deleted in the mean time?
LOG.debug("Attempted to compact a row that doesn't exist.");
} else if (compacted != null) {
// no need to re-compact rows containing a single value.
KeyValue kv = row.get(0);
final byte[] qual = kv.qualifier();
if (qual.length % 2 != 0 || qual.length == 0) {
// This could be a row with only an annotation in it
if ((qual[0] | Annotation.PREFIX()) == Annotation.PREFIX()) {
final Annotation note = JSON.parseToObject(kv.value(),
Annotation.class);
annotations.add(note);
}
return null;
}
final byte[] val = kv.value();
if (qual.length == 2 && Internal.floatingPointValueToFix(qual[1], val)) {
// Fix up old, incorrectly encoded floating point value.
final byte[] newval = Internal.fixFloatingPointValue(qual[1], val);
final byte[] newqual = new byte[] { qual[0],
Internal.fixQualifierFlags(qual[1], newval.length) };
kv = new KeyValue(kv.key(), kv.family(), newqual, newval);
}
compacted[0] = kv;
}
return null;
}
// We know we have at least 2 cells. We need to go through all the cells
// to determine what kind of compaction we're going to do. If each cell
// contains a single individual data point, then we can do a trivial
// compaction. Otherwise, we have a partially compacted row, and the
// logic required to compact it is more complex.
boolean write = true; // Do we need to write a compacted cell?
final KeyValue compact;
{
boolean trivial = true; // Are we doing a trivial compaction?
boolean ms_in_row = false;
boolean s_in_row = false;
int qual_len = 0; // Pre-compute the size of the qualifier we'll need.
int val_len = 1; // Reserve an extra byte for meta-data.
KeyValue longest = row.get(0); // KV with the longest qualifier.
int longest_idx = 0; // Index of `longest'.
int nkvs = row.size();
for (int i = 0; i < nkvs; i++) {
final KeyValue kv = row.get(i);
final byte[] qual = kv.qualifier();
// If the qualifier length isn't 2, this row might have already
// been compacted, potentially partially, so we need to merge the
// partially compacted set of cells, with the rest.
final int len = qual.length;
if (len != 2 && len != 4) {
// Datapoints and compacted columns should have qualifiers with an
// even number of bytes. If we find one with an odd number, or an
// empty qualifier (which is possible), we need to remove it from the
// compaction queue.
if (len % 2 != 0 || len == 0) {
// if the qualifier is 3 bytes and starts with the Annotation prefix,
// parse it out.
if ((qual[0] | Annotation.PREFIX()) == Annotation.PREFIX()) {
final Annotation note = JSON.parseToObject(kv.value(),
Annotation.class);
annotations.add(note);
}
row.remove(i); // This is O(n) but should happen *very* rarely.
nkvs--;
i--;
continue;
}
trivial = false;
// We only do this here because no qualifier can be < 2 bytes.
if (len > longest.qualifier().length) {
longest = kv;
longest_idx = i;
}
// we need to check the value meta flag to see if the already compacted
// column has a mixture of second and millisecond timestamps
if ((kv.value()[kv.value().length - 1] & Const.MS_MIXED_COMPACT) ==
Const.MS_MIXED_COMPACT) {
ms_in_row = s_in_row = true;
}
} else {
if (Internal.inMilliseconds(qual[0])) {
ms_in_row = true;
} else {
s_in_row = true;
}
if (len > longest.qualifier().length) {
longest = kv;
longest_idx = i;
}
// there may be a situation where two second columns are concatenated
// into 4 bytes. If so, we need to perform a complex compaction
if (len == 4) {
if (!Internal.inMilliseconds(qual[0])) {
trivial = false;
}
val_len += kv.value().length;
} else {
// We don't need it below for complex compactions, so we update it
// only here in the `else' branch.
final byte[] v = kv.value();
val_len += Internal.floatingPointValueToFix(qual[1], v) ? 4 : v.length;
}
}
qual_len += len;
}
if (row.size() < 2) {
// We got here because we started off with at least 2 KV, but we
// chose to ignore some in the mean time, so now we're left with
// either none, or just one.
if (row.isEmpty()) {
return null; // No KV left, just ignore this whole row.
} // else: row.size() == 1
// We have only one KV left, we call ourselves recursively to handle
// the case where this KV is an old, incorrectly encoded floating
// point value that needs to be fixed. This is guaranteed to not
// recurse again.
return compact(row, compacted, annotations);
} else if (trivial) {
trivial_compactions.incrementAndGet();
compact = trivialCompact(row, qual_len, val_len, (ms_in_row && s_in_row));
} else {
complex_compactions.incrementAndGet();
compact = complexCompact(row, qual_len / 2, (ms_in_row && s_in_row));
// Now it's vital that we check whether the compact KV has the same
// qualifier as one of the qualifiers that were already in the row.
// Otherwise we might do a `put' in this cell, followed by a delete.
// We don't want to delete what we just wrote.
// This can happen if this row was already compacted but someone
// wrote another individual data point at the same timestamp.
// Optimization: since we kept track of which KV had the longest
// qualifier, we can opportunistically check here if it happens to
// have the same qualifier as the one we just created.
final byte[] qual = compact.qualifier();
final byte[] longest_qual = longest.qualifier();
if (qual.length <= longest_qual.length) {
KeyValue dup = null;
int dup_idx = -1;
if (Bytes.equals(longest_qual, qual)) {
dup = longest;
dup_idx = longest_idx;
} else {
// Worst case: to be safe we have to loop again and check all
// the qualifiers and make sure we're not going to overwrite
// anything.
// TODO(tsuna): Try to write a unit test that triggers this code
// path. I'm not even sure it's possible. Should we replace
// this code with an `assert false: "should never be here"'?
for (int i = 0; i < nkvs; i++) {
final KeyValue kv = row.get(i);
if (Bytes.equals(kv.qualifier(), qual)) {
dup = kv;
dup_idx = i;
break;
}
}