|
@@ -15,14 +15,8 @@
|
|
|
* See the License for the specific language governing permissions and
|
|
|
* limitations under the License.
|
|
|
*/
|
|
|
-package org.apache.hadoop.yarn.server.timelineservice.storage.common;
|
|
|
|
|
|
-import java.io.IOException;
|
|
|
-import java.util.HashMap;
|
|
|
-import java.util.Map;
|
|
|
-import java.util.Map.Entry;
|
|
|
-import java.util.NavigableMap;
|
|
|
-import java.util.TreeMap;
|
|
|
+package org.apache.hadoop.yarn.server.timelineservice.storage.common;
|
|
|
|
|
|
import org.apache.hadoop.hbase.client.Put;
|
|
|
import org.apache.hadoop.hbase.client.Result;
|
|
@@ -32,56 +26,89 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
+import java.io.IOException;
|
|
|
+import java.util.HashMap;
|
|
|
+import java.util.Map;
|
|
|
+import java.util.NavigableMap;
|
|
|
+import java.util.TreeMap;
|
|
|
+
|
|
|
/**
|
|
|
- * This class is meant to be used only by explicit Columns, and not directly to
|
|
|
- * write by clients.
|
|
|
- *
|
|
|
- * @param <T> refers to the table.
|
|
|
+ * A set of utility functions that read or read to a column.
|
|
|
+ * This class is meant to be used only by explicit Columns,
|
|
|
+ * and not directly to write by clients.
|
|
|
*/
|
|
|
-public class ColumnHelper<T> {
|
|
|
+public final class ColumnRWHelper {
|
|
|
private static final Logger LOG =
|
|
|
LoggerFactory.getLogger(ColumnHelper.class);
|
|
|
|
|
|
- private final ColumnFamily<T> columnFamily;
|
|
|
+ private ColumnRWHelper() {
|
|
|
+ }
|
|
|
|
|
|
/**
|
|
|
- * Local copy of bytes representation of columnFamily so that we can avoid
|
|
|
- * cloning a new copy over and over.
|
|
|
+ * Figures out the cell timestamp used in the Put For storing.
|
|
|
+ * Will supplement the timestamp if required. Typically done for flow run
|
|
|
+ * table.If we supplement the timestamp, we left shift the timestamp and
|
|
|
+ * supplement it with the AppId id so that there are no collisions in the flow
|
|
|
+ * run table's cells.
|
|
|
*/
|
|
|
- private final byte[] columnFamilyBytes;
|
|
|
-
|
|
|
- private final ValueConverter converter;
|
|
|
-
|
|
|
- private final boolean supplementTs;
|
|
|
-
|
|
|
- public ColumnHelper(ColumnFamily<T> columnFamily) {
|
|
|
- this(columnFamily, GenericConverter.getInstance());
|
|
|
+ private static long getPutTimestamp(
|
|
|
+ Long timestamp, boolean supplementTs, Attribute[] attributes) {
|
|
|
+ if (timestamp == null) {
|
|
|
+ timestamp = System.currentTimeMillis();
|
|
|
+ }
|
|
|
+ if (!supplementTs) {
|
|
|
+ return timestamp;
|
|
|
+ } else {
|
|
|
+ String appId = getAppIdFromAttributes(attributes);
|
|
|
+ long supplementedTS = TimestampGenerator.getSupplementedTimestamp(
|
|
|
+ timestamp, appId);
|
|
|
+ return supplementedTS;
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
- public ColumnHelper(ColumnFamily<T> columnFamily, ValueConverter converter) {
|
|
|
- this(columnFamily, converter, false);
|
|
|
+ private static String getAppIdFromAttributes(Attribute[] attributes) {
|
|
|
+ if (attributes == null) {
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ String appId = null;
|
|
|
+ for (Attribute attribute : attributes) {
|
|
|
+ if (AggregationCompactionDimension.APPLICATION_ID.toString().equals(
|
|
|
+ attribute.getName())) {
|
|
|
+ appId = Bytes.toString(attribute.getValue());
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return appId;
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * @param columnFamily column family implementation.
|
|
|
- * @param converter converter use to encode/decode values stored in the column
|
|
|
- * or column prefix.
|
|
|
- * @param needSupplementTs flag to indicate if cell timestamp needs to be
|
|
|
- * modified for this column by calling
|
|
|
- * {@link TimestampGenerator#getSupplementedTimestamp(long, String)}. This
|
|
|
- * would be required for columns(such as metrics in flow run table) where
|
|
|
- * potential collisions can occur due to same timestamp.
|
|
|
+ * Sends a Mutation to the table. The mutations will be buffered and sent over
|
|
|
+ * the wire as part of a batch.
|
|
|
+ *
|
|
|
+ * @param rowKey
|
|
|
+ * identifying the row to write. Nothing gets written when null.
|
|
|
+ * @param tableMutator
|
|
|
+ * used to modify the underlying HBase table
|
|
|
+ * @param column the column that is to be modified
|
|
|
+ * @param timestamp
|
|
|
+ * version timestamp. When null the current timestamp multiplied with
|
|
|
+ * TimestampGenerator.TS_MULTIPLIER and added with last 3 digits of
|
|
|
+ * app id will be used
|
|
|
+ * @param inputValue
|
|
|
+ * the value to write to the rowKey and column qualifier. Nothing
|
|
|
+ * gets written when null.
|
|
|
+ * @param attributes Attributes to be set for HBase Put.
|
|
|
+ * @throws IOException if any problem occurs during store operation(sending
|
|
|
+ * mutation to table).
|
|
|
*/
|
|
|
- public ColumnHelper(ColumnFamily<T> columnFamily, ValueConverter converter,
|
|
|
- boolean needSupplementTs) {
|
|
|
- this.columnFamily = columnFamily;
|
|
|
- columnFamilyBytes = columnFamily.getBytes();
|
|
|
- if (converter == null) {
|
|
|
- this.converter = GenericConverter.getInstance();
|
|
|
- } else {
|
|
|
- this.converter = converter;
|
|
|
- }
|
|
|
- this.supplementTs = needSupplementTs;
|
|
|
+ public static void store(byte[] rowKey, TypedBufferedMutator<?> tableMutator,
|
|
|
+ Column<?> column, Long timestamp,
|
|
|
+ Object inputValue, Attribute... attributes)
|
|
|
+ throws IOException {
|
|
|
+ store(rowKey, tableMutator, column.getColumnFamilyBytes(),
|
|
|
+ column.getColumnQualifierBytes(), timestamp,
|
|
|
+ column.supplementCellTimestamp(), inputValue,
|
|
|
+ column.getValueConverter(),
|
|
|
+ column.getCombinedAttrsWithAggr(attributes));
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -92,6 +119,7 @@ public class ColumnHelper<T> {
|
|
|
* identifying the row to write. Nothing gets written when null.
|
|
|
* @param tableMutator
|
|
|
* used to modify the underlying HBase table
|
|
|
+ * @param columnFamilyBytes
|
|
|
* @param columnQualifier
|
|
|
* column qualifier. Nothing gets written when null.
|
|
|
* @param timestamp
|
|
@@ -101,18 +129,20 @@ public class ColumnHelper<T> {
|
|
|
* @param inputValue
|
|
|
* the value to write to the rowKey and column qualifier. Nothing
|
|
|
* gets written when null.
|
|
|
+ * @param converter
|
|
|
* @param attributes Attributes to be set for HBase Put.
|
|
|
* @throws IOException if any problem occurs during store operation(sending
|
|
|
* mutation to table).
|
|
|
*/
|
|
|
- public void store(byte[] rowKey, TypedBufferedMutator<?> tableMutator,
|
|
|
- byte[] columnQualifier, Long timestamp, Object inputValue,
|
|
|
+ public static void store(byte[] rowKey, TypedBufferedMutator<?> tableMutator,
|
|
|
+ byte[] columnFamilyBytes, byte[] columnQualifier, Long timestamp,
|
|
|
+ boolean supplementTs, Object inputValue, ValueConverter converter,
|
|
|
Attribute... attributes) throws IOException {
|
|
|
if ((rowKey == null) || (columnQualifier == null) || (inputValue == null)) {
|
|
|
return;
|
|
|
}
|
|
|
Put p = new Put(rowKey);
|
|
|
- timestamp = getPutTimestamp(timestamp, attributes);
|
|
|
+ timestamp = getPutTimestamp(timestamp, supplementTs, attributes);
|
|
|
p.addColumn(columnFamilyBytes, columnQualifier, timestamp,
|
|
|
converter.encodeValue(inputValue));
|
|
|
if ((attributes != null) && (attributes.length > 0)) {
|
|
@@ -123,59 +153,20 @@ public class ColumnHelper<T> {
|
|
|
tableMutator.mutate(p);
|
|
|
}
|
|
|
|
|
|
- /*
|
|
|
- * Figures out the cell timestamp used in the Put For storing.
|
|
|
- * Will supplement the timestamp if required. Typically done for flow run
|
|
|
- * table.If we supplement the timestamp, we left shift the timestamp and
|
|
|
- * supplement it with the AppId id so that there are no collisions in the flow
|
|
|
- * run table's cells.
|
|
|
- */
|
|
|
- private long getPutTimestamp(Long timestamp, Attribute[] attributes) {
|
|
|
- if (timestamp == null) {
|
|
|
- timestamp = System.currentTimeMillis();
|
|
|
- }
|
|
|
- if (!this.supplementTs) {
|
|
|
- return timestamp;
|
|
|
- } else {
|
|
|
- String appId = getAppIdFromAttributes(attributes);
|
|
|
- long supplementedTS = TimestampGenerator.getSupplementedTimestamp(
|
|
|
- timestamp, appId);
|
|
|
- return supplementedTS;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private String getAppIdFromAttributes(Attribute[] attributes) {
|
|
|
- if (attributes == null) {
|
|
|
- return null;
|
|
|
- }
|
|
|
- String appId = null;
|
|
|
- for (Attribute attribute : attributes) {
|
|
|
- if (AggregationCompactionDimension.APPLICATION_ID.toString().equals(
|
|
|
- attribute.getName())) {
|
|
|
- appId = Bytes.toString(attribute.getValue());
|
|
|
- }
|
|
|
- }
|
|
|
- return appId;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * @return the column family for this column implementation.
|
|
|
- */
|
|
|
- public ColumnFamily<T> getColumnFamily() {
|
|
|
- return columnFamily;
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Get the latest version of this specified column. Note: this call clones the
|
|
|
* value content of the hosting {@link org.apache.hadoop.hbase.Cell Cell}.
|
|
|
*
|
|
|
* @param result from which to read the value. Cannot be null
|
|
|
+ * @param columnFamilyBytes
|
|
|
* @param columnQualifierBytes referring to the column to be read.
|
|
|
+ * @param converter
|
|
|
* @return latest version of the specified column of whichever object was
|
|
|
* written.
|
|
|
* @throws IOException if any problem occurs while reading result.
|
|
|
*/
|
|
|
- public Object readResult(Result result, byte[] columnQualifierBytes)
|
|
|
+ public static Object readResult(Result result, byte[] columnFamilyBytes,
|
|
|
+ byte[] columnQualifierBytes, ValueConverter converter)
|
|
|
throws IOException {
|
|
|
if (result == null || columnQualifierBytes == null) {
|
|
|
return null;
|
|
@@ -188,6 +179,87 @@ public class ColumnHelper<T> {
|
|
|
return converter.decodeValue(value);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Get the latest version of this specified column. Note: this call clones the
|
|
|
+ * value content of the hosting {@link org.apache.hadoop.hbase.Cell Cell}.
|
|
|
+ *
|
|
|
+ * @param result from which to read the value. Cannot be null
|
|
|
+ * @param column the column that the result can be parsed to
|
|
|
+ * @return latest version of the specified column of whichever object was
|
|
|
+ * written.
|
|
|
+ * @throws IOException if any problem occurs while reading result.
|
|
|
+ */
|
|
|
+ public static Object readResult(Result result, Column<?> column)
|
|
|
+ throws IOException {
|
|
|
+ return readResult(result, column.getColumnFamilyBytes(),
|
|
|
+ column.getColumnQualifierBytes(), column.getValueConverter());
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Get the latest version of this specified column. Note: this call clones the
|
|
|
+ * value content of the hosting {@link org.apache.hadoop.hbase.Cell Cell}.
|
|
|
+ *
|
|
|
+ * @param result Cannot be null
|
|
|
+ * @param columnPrefix column prefix to read from
|
|
|
+ * @param qualifier column qualifier. Nothing gets read when null.
|
|
|
+ * @return result object (can be cast to whatever object was written to) or
|
|
|
+ * null when specified column qualifier for this prefix doesn't exist
|
|
|
+ * in the result.
|
|
|
+ * @throws IOException if there is any exception encountered while reading
|
|
|
+ * result.
|
|
|
+ */
|
|
|
+ public static Object readResult(Result result, ColumnPrefix<?> columnPrefix,
|
|
|
+ String qualifier) throws IOException {
|
|
|
+ byte[] columnQualifier = ColumnHelper.getColumnQualifier(
|
|
|
+ columnPrefix.getColumnPrefixInBytes(), qualifier);
|
|
|
+
|
|
|
+ return readResult(
|
|
|
+ result, columnPrefix.getColumnFamilyBytes(),
|
|
|
+ columnQualifier, columnPrefix.getValueConverter());
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ *
|
|
|
+ * @param <K> identifies the type of key converter.
|
|
|
+ * @param result from which to read columns.
|
|
|
+ * @param keyConverter used to convert column bytes to the appropriate key
|
|
|
+ * type
|
|
|
+ * @return the latest values of columns in the column family with this prefix
|
|
|
+ * (or all of them if the prefix value is null).
|
|
|
+ * @throws IOException if there is any exception encountered while reading
|
|
|
+ * results.
|
|
|
+ */
|
|
|
+ public static <K> Map<K, Object> readResults(Result result,
|
|
|
+ ColumnPrefix<?> columnPrefix, KeyConverter<K> keyConverter)
|
|
|
+ throws IOException {
|
|
|
+ return readResults(result,
|
|
|
+ columnPrefix.getColumnFamilyBytes(),
|
|
|
+ columnPrefix.getColumnPrefixInBytes(),
|
|
|
+ keyConverter, columnPrefix.getValueConverter());
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * @param result from which to reads data with timestamps.
|
|
|
+ * @param <K> identifies the type of key converter.
|
|
|
+ * @param <V> the type of the values. The values will be cast into that type.
|
|
|
+ * @param keyConverter used to convert column bytes to the appropriate key
|
|
|
+ * type.
|
|
|
+ * @return the cell values at each respective time in for form
|
|
|
+ * {@literal {idA={timestamp1->value1}, idA={timestamp2->value2},
|
|
|
+ * idB={timestamp3->value3}, idC={timestamp1->value4}}}
|
|
|
+ * @throws IOException if there is any exception encountered while reading
|
|
|
+ * result.
|
|
|
+ */
|
|
|
+ public static <K, V> NavigableMap<K, NavigableMap<Long, V>>
|
|
|
+ readResultsWithTimestamps(Result result, ColumnPrefix<?> columnPrefix,
|
|
|
+ KeyConverter<K> keyConverter) throws IOException {
|
|
|
+ return readResultsWithTimestamps(result,
|
|
|
+ columnPrefix.getColumnFamilyBytes(),
|
|
|
+ columnPrefix.getColumnPrefixInBytes(),
|
|
|
+ keyConverter, columnPrefix.getValueConverter(),
|
|
|
+ columnPrefix.supplementCellTimeStamp());
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* @param result from which to reads data with timestamps
|
|
|
* @param columnPrefixBytes optional prefix to limit columns. If null all
|
|
@@ -203,22 +275,24 @@ public class ColumnHelper<T> {
|
|
|
* @throws IOException if any problem occurs while reading results.
|
|
|
*/
|
|
|
@SuppressWarnings("unchecked")
|
|
|
- public <K, V> NavigableMap<K, NavigableMap<Long, V>>
|
|
|
- readResultsWithTimestamps(Result result, byte[] columnPrefixBytes,
|
|
|
- KeyConverter<K> keyConverter) throws IOException {
|
|
|
+ public static <K, V> NavigableMap<K, NavigableMap<Long, V>>
|
|
|
+ readResultsWithTimestamps(Result result, byte[] columnFamilyBytes,
|
|
|
+ byte[] columnPrefixBytes, KeyConverter<K> keyConverter,
|
|
|
+ ValueConverter valueConverter, boolean supplementTs)
|
|
|
+ throws IOException {
|
|
|
|
|
|
NavigableMap<K, NavigableMap<Long, V>> results = new TreeMap<>();
|
|
|
|
|
|
if (result != null) {
|
|
|
NavigableMap<
|
|
|
byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>> resultMap =
|
|
|
- result.getMap();
|
|
|
+ result.getMap();
|
|
|
|
|
|
NavigableMap<byte[], NavigableMap<Long, byte[]>> columnCellMap =
|
|
|
resultMap.get(columnFamilyBytes);
|
|
|
// could be that there is no such column family.
|
|
|
if (columnCellMap != null) {
|
|
|
- for (Entry<byte[], NavigableMap<Long, byte[]>> entry : columnCellMap
|
|
|
+ for (Map.Entry<byte[], NavigableMap<Long, byte[]>> entry : columnCellMap
|
|
|
.entrySet()) {
|
|
|
K converterColumnKey = null;
|
|
|
if (columnPrefixBytes == null) {
|
|
@@ -255,9 +329,9 @@ public class ColumnHelper<T> {
|
|
|
new TreeMap<Long, V>();
|
|
|
NavigableMap<Long, byte[]> cells = entry.getValue();
|
|
|
if (cells != null) {
|
|
|
- for (Entry<Long, byte[]> cell : cells.entrySet()) {
|
|
|
+ for (Map.Entry<Long, byte[]> cell : cells.entrySet()) {
|
|
|
V value =
|
|
|
- (V) converter.decodeValue(cell.getValue());
|
|
|
+ (V) valueConverter.decodeValue(cell.getValue());
|
|
|
Long ts = supplementTs ? TimestampGenerator.
|
|
|
getTruncatedTimestamp(cell.getKey()) : cell.getKey();
|
|
|
cellResults.put(ts, value);
|
|
@@ -286,14 +360,15 @@ public class ColumnHelper<T> {
|
|
|
* returning byte arrays of values that were not Strings.
|
|
|
* @throws IOException if any problem occurs while reading results.
|
|
|
*/
|
|
|
- public <K> Map<K, Object> readResults(Result result,
|
|
|
- byte[] columnPrefixBytes, KeyConverter<K> keyConverter)
|
|
|
+ public static <K> Map<K, Object> readResults(Result result,
|
|
|
+ byte[] columnFamilyBytes, byte[] columnPrefixBytes,
|
|
|
+ KeyConverter<K> keyConverter, ValueConverter valueConverter)
|
|
|
throws IOException {
|
|
|
Map<K, Object> results = new HashMap<K, Object>();
|
|
|
|
|
|
if (result != null) {
|
|
|
Map<byte[], byte[]> columns = result.getFamilyMap(columnFamilyBytes);
|
|
|
- for (Entry<byte[], byte[]> entry : columns.entrySet()) {
|
|
|
+ for (Map.Entry<byte[], byte[]> entry : columns.entrySet()) {
|
|
|
byte[] columnKey = entry.getKey();
|
|
|
if (columnKey != null && columnKey.length > 0) {
|
|
|
|
|
@@ -327,7 +402,7 @@ public class ColumnHelper<T> {
|
|
|
// If the columnPrefix is null (we want all columns), or the actual
|
|
|
// prefix matches the given prefix we want this column
|
|
|
if (converterColumnKey != null) {
|
|
|
- Object value = converter.decodeValue(entry.getValue());
|
|
|
+ Object value = valueConverter.decodeValue(entry.getValue());
|
|
|
// we return the columnQualifier in parts since we don't know
|
|
|
// which part is of which data type.
|
|
|
results.put(converterColumnKey, value);
|
|
@@ -339,76 +414,74 @@ public class ColumnHelper<T> {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * @param columnPrefixBytes The byte representation for the column prefix.
|
|
|
- * Should not contain {@link Separator#QUALIFIERS}.
|
|
|
- * @param qualifier for the remainder of the column.
|
|
|
- * {@link Separator#QUALIFIERS} is permissible in the qualifier
|
|
|
- * as it is joined only with the column prefix bytes.
|
|
|
- * @return fully sanitized column qualifier that is a combination of prefix
|
|
|
- * and qualifier. If prefix is null, the result is simply the encoded
|
|
|
- * qualifier without any separator.
|
|
|
- */
|
|
|
- public static byte[] getColumnQualifier(byte[] columnPrefixBytes,
|
|
|
- String qualifier) {
|
|
|
-
|
|
|
- // We don't want column names to have spaces / tabs.
|
|
|
- byte[] encodedQualifier =
|
|
|
- Separator.encode(qualifier, Separator.SPACE, Separator.TAB);
|
|
|
- if (columnPrefixBytes == null) {
|
|
|
- return encodedQualifier;
|
|
|
- }
|
|
|
-
|
|
|
- // Convert qualifier to lower case, strip of separators and tag on column
|
|
|
- // prefix.
|
|
|
- byte[] columnQualifier =
|
|
|
- Separator.QUALIFIERS.join(columnPrefixBytes, encodedQualifier);
|
|
|
- return columnQualifier;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * @param columnPrefixBytes The byte representation for the column prefix.
|
|
|
- * Should not contain {@link Separator#QUALIFIERS}.
|
|
|
- * @param qualifier for the remainder of the column.
|
|
|
- * @return fully sanitized column qualifier that is a combination of prefix
|
|
|
- * and qualifier. If prefix is null, the result is simply the encoded
|
|
|
- * qualifier without any separator.
|
|
|
+ * Sends a Mutation to the table. The mutations will be buffered and sent over
|
|
|
+ * the wire as part of a batch.
|
|
|
+ *
|
|
|
+ * @param rowKey identifying the row to write. Nothing gets written when null.
|
|
|
+ * @param tableMutator used to modify the underlying HBase table. Caller is
|
|
|
+ * responsible to pass a mutator for the table that actually has this
|
|
|
+ * column.
|
|
|
+ * @param qualifier column qualifier. Nothing gets written when null.
|
|
|
+ * @param timestamp version timestamp. When null the server timestamp will be
|
|
|
+ * used.
|
|
|
+ * @param attributes attributes for the mutation that are used by the
|
|
|
+ * coprocessor to set/read the cell tags.
|
|
|
+ * @param inputValue the value to write to the rowKey and column qualifier.
|
|
|
+ * Nothing gets written when null.
|
|
|
+ * @throws IOException if there is any exception encountered while doing
|
|
|
+ * store operation(sending mutation to the table).
|
|
|
*/
|
|
|
- public static byte[] getColumnQualifier(byte[] columnPrefixBytes,
|
|
|
- long qualifier) {
|
|
|
-
|
|
|
- if (columnPrefixBytes == null) {
|
|
|
- return Bytes.toBytes(qualifier);
|
|
|
+ public static void store(byte[] rowKey, TypedBufferedMutator<?> tableMutator,
|
|
|
+ ColumnPrefix<?> columnPrefix, byte[] qualifier, Long timestamp,
|
|
|
+ Object inputValue, Attribute... attributes) throws IOException {
|
|
|
+ // Null check
|
|
|
+ if (qualifier == null) {
|
|
|
+ throw new IOException("Cannot store column with null qualifier in "
|
|
|
+ +tableMutator.getName().getNameAsString());
|
|
|
}
|
|
|
|
|
|
- // Convert qualifier to lower case, strip of separators and tag on column
|
|
|
- // prefix.
|
|
|
- byte[] columnQualifier =
|
|
|
- Separator.QUALIFIERS.join(columnPrefixBytes, Bytes.toBytes(qualifier));
|
|
|
- return columnQualifier;
|
|
|
- }
|
|
|
+ byte[] columnQualifier = columnPrefix.getColumnPrefixBytes(qualifier);
|
|
|
+ Attribute[] combinedAttributes =
|
|
|
+ columnPrefix.getCombinedAttrsWithAggr(attributes);
|
|
|
|
|
|
- public ValueConverter getValueConverter() {
|
|
|
- return converter;
|
|
|
+ store(rowKey, tableMutator, columnPrefix.getColumnFamilyBytes(),
|
|
|
+ columnQualifier, timestamp, columnPrefix.supplementCellTimeStamp(),
|
|
|
+ inputValue, columnPrefix.getValueConverter(), combinedAttributes);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * @param columnPrefixBytes The byte representation for the column prefix.
|
|
|
- * Should not contain {@link Separator#QUALIFIERS}.
|
|
|
- * @param qualifier the byte representation for the remainder of the column.
|
|
|
- * @return fully sanitized column qualifier that is a combination of prefix
|
|
|
- * and qualifier. If prefix is null, the result is simply the encoded
|
|
|
- * qualifier without any separator.
|
|
|
+ * Sends a Mutation to the table. The mutations will be buffered and sent over
|
|
|
+ * the wire as part of a batch.
|
|
|
+ *
|
|
|
+ * @param rowKey identifying the row to write. Nothing gets written when null.
|
|
|
+ * @param tableMutator used to modify the underlying HBase table. Caller is
|
|
|
+ * responsible to pass a mutator for the table that actually has this
|
|
|
+ * column.
|
|
|
+ * @param qualifier column qualifier. Nothing gets written when null.
|
|
|
+ * @param timestamp version timestamp. When null the server timestamp will be
|
|
|
+ * used.
|
|
|
+ * @param attributes attributes for the mutation that are used by the
|
|
|
+ * coprocessor to set/read the cell tags.
|
|
|
+ * @param inputValue the value to write to the rowKey and column qualifier.
|
|
|
+ * Nothing gets written when null.
|
|
|
+ * @throws IOException if there is any exception encountered while doing
|
|
|
+ * store operation(sending mutation to the table).
|
|
|
*/
|
|
|
- public static byte[] getColumnQualifier(byte[] columnPrefixBytes,
|
|
|
- byte[] qualifier) {
|
|
|
-
|
|
|
- if (columnPrefixBytes == null) {
|
|
|
- return qualifier;
|
|
|
+ public static void store(byte[] rowKey, TypedBufferedMutator<?> tableMutator,
|
|
|
+ ColumnPrefix<?> columnPrefix, String qualifier, Long timestamp,
|
|
|
+ Object inputValue, Attribute... attributes) throws IOException {
|
|
|
+ // Null check
|
|
|
+ if (qualifier == null) {
|
|
|
+ throw new IOException("Cannot store column with null qualifier in "
|
|
|
+ + tableMutator.getName().getNameAsString());
|
|
|
}
|
|
|
|
|
|
- byte[] columnQualifier =
|
|
|
- Separator.QUALIFIERS.join(columnPrefixBytes, qualifier);
|
|
|
- return columnQualifier;
|
|
|
- }
|
|
|
+ byte[] columnQualifier = columnPrefix.getColumnPrefixBytes(qualifier);
|
|
|
+ Attribute[] combinedAttributes =
|
|
|
+ columnPrefix.getCombinedAttrsWithAggr(attributes);
|
|
|
|
|
|
+ store(rowKey, tableMutator, columnPrefix.getColumnFamilyBytes(),
|
|
|
+ columnQualifier, timestamp, columnPrefix.supplementCellTimeStamp(),
|
|
|
+ inputValue, columnPrefix.getValueConverter(), combinedAttributes);
|
|
|
+ }
|
|
|
}
|