http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongConverter.java ---------------------------------------------------------------------- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongConverter.java new file mode 100644 index 0000000..0857980 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongConverter.java @@ -0,0 +1,94 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * 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.io.Serializable; + +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Encodes a value by interpreting it as a Long and converting it to bytes and + * decodes a set of bytes as a Long. + */ +public final class LongConverter implements NumericValueConverter, + Serializable { + + /** + * Added because we implement Comparator<Number>. + */ + private static final long serialVersionUID = 1L; + + public LongConverter() { + } + + @Override + public byte[] encodeValue(Object value) throws IOException { + if (!HBaseTimelineSchemaUtils.isIntegralValue(value)) { + throw new IOException("Expected integral value"); + } + return Bytes.toBytes(((Number)value).longValue()); + } + + @Override + public Object decodeValue(byte[] bytes) throws IOException { + if (bytes == null) { + return null; + } + return Bytes.toLong(bytes); + } + + /** + * Compares two numbers as longs. If either number is null, it will be taken + * as 0. + * + * @param num1 the first {@code Long} to compare. + * @param num2 the second {@code Long} to compare. + * @return -1 if num1 is less than num2, 0 if num1 is equal to num2 and 1 if + * num1 is greater than num2. + */ + @Override + public int compare(Number num1, Number num2) { + return Long.compare((num1 == null) ? 0L : num1.longValue(), + (num2 == null) ? 0L : num2.longValue()); + } + + @Override + public Number add(Number num1, Number num2, Number...numbers) { + long sum = ((num1 == null) ? 0L : num1.longValue()) + + ((num2 == null) ? 0L : num2.longValue()); + for (Number num : numbers) { + sum = sum + ((num == null) ? 0L : num.longValue()); + } + return sum; + } + + /** + * Converts a timestamp into it's inverse timestamp to be used in (row) keys + * where we want to have the most recent timestamp in the top of the table + * (scans start at the most recent timestamp first). + * + * @param key value to be inverted so that the latest version will be first in + * a scan. + * @return inverted long + */ + public static long invertLong(long key) { + return Long.MAX_VALUE - key; + } +}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongKeyConverter.java ---------------------------------------------------------------------- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongKeyConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongKeyConverter.java new file mode 100644 index 0000000..4a724d6 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/LongKeyConverter.java @@ -0,0 +1,68 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * 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; +/** + * Encodes and decodes column names / row keys which are long. + */ +public final class LongKeyConverter implements KeyConverter<Long> { + + /** + * To delegate the actual work to. + */ + private final LongConverter longConverter = new LongConverter(); + + public LongKeyConverter() { + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter + * #encode(java.lang.Object) + */ + @Override + public byte[] encode(Long key) { + try { + // IOException will not be thrown here as we are explicitly passing + // Long. + return longConverter.encodeValue(key); + } catch (IOException e) { + return null; + } + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter + * #decode(byte[]) + */ + @Override + public Long decode(byte[] bytes) { + try { + return (Long) longConverter.decodeValue(bytes); + } catch (IOException e) { + return null; + } + } +} http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/NumericValueConverter.java ---------------------------------------------------------------------- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/NumericValueConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/NumericValueConverter.java new file mode 100644 index 0000000..8fb6536 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/NumericValueConverter.java @@ -0,0 +1,39 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * 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.util.Comparator; + +/** + * Extends ValueConverter interface for numeric converters to support numerical + * operations such as comparison, addition, etc. + */ +public interface NumericValueConverter extends ValueConverter, + Comparator<Number> { + /** + * Adds two or more numbers. If either of the numbers are null, it is taken as + * 0. + * + * @param num1 the first number to add. + * @param num2 the second number to add. + * @param numbers Rest of the numbers to be added. + * @return result after adding up the numbers. + */ + Number add(Number num1, Number num2, Number...numbers); +} http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Range.java ---------------------------------------------------------------------- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Range.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Range.java new file mode 100644 index 0000000..8a2e01a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Range.java @@ -0,0 +1,62 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Encapsulates a range with start and end indices. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public class Range { + private final int startIdx; + private final int endIdx; + + /** + * Defines a range from start index (inclusive) to end index (exclusive). + * + * @param start + * Starting index position + * @param end + * Ending index position (exclusive) + */ + public Range(int start, int end) { + if (start < 0 || end < start) { + throw new IllegalArgumentException( + "Invalid range, required that: 0 <= start <= end; start=" + start + + ", end=" + end); + } + + this.startIdx = start; + this.endIdx = end; + } + + public int start() { + return startIdx; + } + + public int end() { + return endIdx; + } + + public int length() { + return endIdx - startIdx; + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/RowKeyPrefix.java ---------------------------------------------------------------------- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/RowKeyPrefix.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/RowKeyPrefix.java new file mode 100644 index 0000000..6159dc7 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/RowKeyPrefix.java @@ -0,0 +1,42 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +/** + * In queries where a single result is needed, an exact rowkey can be used + * through the corresponding rowkey#getRowKey() method. For queries that need to + * scan over a range of rowkeys, a partial (the initial part) of rowkeys are + * used. Classes implementing RowKeyPrefix indicate that they are the initial + * part of rowkeys, with different constructors with fewer number of argument to + * form a partial rowkey, a prefix. + * + * @param <R> indicating the type of rowkey that a particular implementation is + * a prefix for. + */ +public interface RowKeyPrefix<R> { + + /** + * Create a row key prefix, meaning a partial rowkey that can be used in range + * scans. Which fields are included in the prefix will depend on the + * constructor of the specific instance that was used. Output depends on which + * constructor was used. + * @return a prefix of the following form {@code fist!second!...!last!} + */ + byte[] getRowKeyPrefix(); + +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Separator.java ---------------------------------------------------------------------- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Separator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Separator.java new file mode 100644 index 0000000..5090b4d --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/Separator.java @@ -0,0 +1,575 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 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.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Used to separate row qualifiers, column qualifiers and compound fields. + */ +public enum Separator { + + /** + * separator in key or column qualifier fields. + */ + QUALIFIERS("!", "%0$"), + + /** + * separator in values, and/or compound key/column qualifier fields. + */ + VALUES("=", "%1$"), + + /** + * separator in values, often used to avoid having these in qualifiers and + * names. Note that if we use HTML form encoding through URLEncoder, we end up + * getting a + for a space, which may already occur in strings, so we don't + * want that. + */ + SPACE(" ", "%2$"), + + /** + * separator in values, often used to avoid having these in qualifiers and + * names. + */ + TAB("\t", "%3$"); + + // a reserved character that starts each of the encoded values and is encoded + // first in order to escape naturally occurring instances of encoded values + // although it can be expressed as an enum instance, we define them as private + // variables to hide it from callers + private static final String PERCENT = "%"; + private static final String PERCENT_ENCODED = "%9$"; + + private static final Pattern PERCENT_PATTERN = + Pattern.compile(PERCENT, Pattern.LITERAL); + private static final String PERCENT_REPLACEMENT = + Matcher.quoteReplacement(PERCENT); + + private static final Pattern PERCENT_ENCODED_PATTERN = + Pattern.compile(PERCENT_ENCODED, Pattern.LITERAL); + private static final String PERCENT_ENCODED_REPLACEMENT = + Matcher.quoteReplacement(PERCENT_ENCODED); + + /** + * The string value of this separator. + */ + private final String value; + + /** + * The bye representation of value. + */ + private final byte[] bytes; + + // pre-compiled patterns and quoted replacements for optimization + private final Pattern valuePattern; + private final String valueReplacement; + + private final Pattern encodedValuePattern; + private final String encodedValueReplacement; + + /** + * Indicator for variable size of an individual segment in a split. The + * segment ends wherever separator is encountered. + * Typically used for string. + * Also used to indicate that there is no fixed number of splits which need to + * be returned. If split limit is specified as this, all possible splits are + * returned. + */ + public static final int VARIABLE_SIZE = 0; + + + /** empty string. */ + public static final String EMPTY_STRING = ""; + + /** empty bytes. */ + public static final byte[] EMPTY_BYTES = new byte[0]; + + /** + * @param value of the separator to use. Cannot be null or empty string. + * @param encodedValue choose something that isn't likely to occur in the data + * itself. Cannot be null or empty string. + */ + private Separator(String value, String encodedValue) { + this.value = value; + + // validation + if (value == null || value.length() == 0 || encodedValue == null + || encodedValue.length() == 0) { + throw new IllegalArgumentException( + "Cannot create separator from null or empty string."); + } + + this.bytes = Bytes.toBytes(value); + this.valuePattern = Pattern.compile(value, Pattern.LITERAL); + this.valueReplacement = Matcher.quoteReplacement(value); + + this.encodedValuePattern = Pattern.compile(encodedValue, Pattern.LITERAL); + this.encodedValueReplacement = Matcher.quoteReplacement(encodedValue); + } + + /** + * @return the original value of the separator + */ + public String getValue() { + return value; + } + + /** + * Used to make token safe to be used with this separator without collisions. + * It <em>must</em> be paired with {@link #decode(String)} for it to be + * decoded correctly. + * <p> + * If you need to encode a given string for multiple separators, + * {@link #encode(String, Separator...)} should be used over successive + * invocations of this method. It will result in a more compact version of the + * encoded value. + * + * @param token Token to be encoded. + * @return the token with any occurrences of this separator URLEncoded. + */ + public String encode(String token) { + if (token == null || token.length() == 0) { + // Nothing to replace + return token; + } + // first encode the percent to escape naturally occurring encoded values + String escaped = encodePercent(token); + return encodeSingle(escaped, this); + } + + private static String replace(String token, Pattern pattern, + String replacement) { + return pattern.matcher(token).replaceAll(replacement); + } + + private static String encodeSingle(String token, Separator separator) { + return replace(token, separator.valuePattern, + separator.encodedValueReplacement); + } + + private static String encodePercent(String token) { + return replace(token, PERCENT_PATTERN, PERCENT_ENCODED_REPLACEMENT); + } + + /** + * Decode the token encoded using {@link #encode(String)}. It <em>must</em> be + * used for the result encoded with {@link #encode(String)} to be able to + * recover the original. + * + * @param token Token to be decoded. + * @return the token with any occurrences of the encoded separator replaced by + * the separator itself. + */ + public String decode(String token) { + if (token == null || token.length() == 0) { + // Nothing to replace + return token; + } + String escaped = decodeSingle(token, this); + // decode percent to de-escape + return decodePercent(escaped); + } + + private static String decodeSingle(String token, Separator separator) { + return replace(token, separator.encodedValuePattern, + separator.valueReplacement); + } + + private static String decodePercent(String token) { + return replace(token, PERCENT_ENCODED_PATTERN, PERCENT_REPLACEMENT); + } + + /** + * Encode the given separators in the token with their encoding equivalents. + * It <em>must</em> be paired with {@link #decode(byte[], Separator...)} or + * {@link #decode(String, Separator...)} with the same separators for it to be + * decoded correctly. + * <p> + * If you need to encode a given string for multiple separators, this form of + * encoding should be used over successive invocations of + * {@link #encode(String)}. It will result in a more compact version of the + * encoded value. + * + * @param token containing possible separators that need to be encoded. + * @param separators to be encoded in the token with their URLEncoding + * equivalent. + * @return non-null byte representation of the token with occurrences of the + * separators encoded. + */ + public static byte[] encode(String token, Separator... separators) { + if (token == null || token.length() == 0) { + return EMPTY_BYTES; + } + String result = token; + // first encode the percent to escape naturally occurring encoded values + result = encodePercent(token); + for (Separator separator : separators) { + if (separator != null) { + result = encodeSingle(result, separator); + } + } + return Bytes.toBytes(result); + } + + /** + * Decode the given separators in the token with their decoding equivalents. + * It <em>must</em> be used for the result encoded with + * {@link #encode(String, Separator...)} with the same separators to be able + * to recover the original. + * + * @param token containing possible separators that need to be encoded. + * @param separators to be encoded in the token with their URLEncoding + * equivalent. + * @return String representation of the token with occurrences of the URL + * encoded separators decoded. + */ + public static String decode(byte[] token, Separator... separators) { + if (token == null) { + return null; + } + return decode(Bytes.toString(token), separators); + } + + /** + * Decode the given separators in the token with their decoding equivalents. + * It <em>must</em> be used for the result encoded with + * {@link #encode(String, Separator...)} with the same separators to be able + * to recover the original. + * + * @param token containing possible separators that need to be encoded. + * @param separators to be encoded in the token with their URLEncoding + * equivalent. + * @return String representation of the token with occurrences of the URL + * encoded separators decoded. + */ + public static String decode(String token, Separator... separators) { + if (token == null) { + return null; + } + String result = token; + for (Separator separator : separators) { + if (separator != null) { + result = decodeSingle(result, separator); + } + } + // decode percent to de-escape + return decodePercent(result); + } + + /** + * Returns a single byte array containing all of the individual arrays + * components separated by this separator. + * + * @param components Byte array components to be joined together. + * @return byte array after joining the components + */ + public byte[] join(byte[]... components) { + if (components == null || components.length == 0) { + return EMPTY_BYTES; + } + + int finalSize = 0; + finalSize = this.value.length() * (components.length - 1); + for (byte[] comp : components) { + if (comp != null) { + finalSize += comp.length; + } + } + + byte[] buf = new byte[finalSize]; + int offset = 0; + for (int i = 0; i < components.length; i++) { + if (components[i] != null) { + System.arraycopy(components[i], 0, buf, offset, components[i].length); + offset += components[i].length; + } + if (i < (components.length - 1)) { + System.arraycopy(this.bytes, 0, buf, offset, this.value.length()); + offset += this.value.length(); + } + } + return buf; + } + + /** + * Concatenates items (as String), using this separator. + * + * @param items Items join, {@code toString()} will be called in each item. + * Any occurrence of the separator in the individual strings will be + * first encoded. Cannot be null. + * @return non-null joined result. Note that when separator is {@literal null} + * the result is simply all items concatenated and the process is not + * reversible through {@link #splitEncoded(String)} + */ + public String joinEncoded(String... items) { + if (items == null || items.length == 0) { + return ""; + } + + StringBuilder sb = new StringBuilder(encode(items[0].toString())); + // Start at 1, we've already grabbed the first value at index 0 + for (int i = 1; i < items.length; i++) { + sb.append(this.value); + sb.append(encode(items[i].toString())); + } + + return sb.toString(); + } + + /** + * Concatenates items (as String), using this separator. + * + * @param items Items join, {@code toString()} will be called in each item. + * Any occurrence of the separator in the individual strings will be + * first encoded. Cannot be null. + * @return non-null joined result. Note that when separator is {@literal null} + * the result is simply all items concatenated and the process is not + * reversible through {@link #splitEncoded(String)} + */ + public String joinEncoded(Iterable<?> items) { + if (items == null) { + return ""; + } + Iterator<?> i = items.iterator(); + if (!i.hasNext()) { + return ""; + } + + StringBuilder sb = new StringBuilder(encode(i.next().toString())); + while (i.hasNext()) { + sb.append(this.value); + sb.append(encode(i.next().toString())); + } + + return sb.toString(); + } + + /** + * @param compoundValue containing individual values separated by this + * separator, which have that separator encoded. + * @return non-null set of values from the compoundValue with the separator + * decoded. + */ + public Collection<String> splitEncoded(String compoundValue) { + List<String> result = new ArrayList<String>(); + if (compoundValue != null) { + for (String val : valuePattern.split(compoundValue)) { + result.add(decode(val)); + } + } + return result; + } + + /** + * Splits the source array into multiple array segments using this separator, + * up to a maximum of count items. This will naturally produce copied byte + * arrays for each of the split segments. + * + * @param source to be split + * @param limit on how many segments are supposed to be returned. A + * non-positive value indicates no limit on number of segments. + * @return source split by this separator. + */ + public byte[][] split(byte[] source, int limit) { + return split(source, this.bytes, limit); + } + + /** + * Splits the source array into multiple array segments using this separator. + * The sizes indicate the sizes of the relative components/segments. + * In case one of the segments contains this separator before the specified + * size is reached, the separator will be considered part of that segment and + * we will continue till size is reached. + * Variable length strings cannot contain this separator and are indiced with + * a size of {@value #VARIABLE_SIZE}. Such strings are encoded for this + * separator and decoded after the results from split is returned. + * + * @param source byte array to be split. + * @param sizes sizes of relative components/segments. + * @return source split by this separator as per the sizes specified.. + */ + public byte[][] split(byte[] source, int[] sizes) { + return split(source, this.bytes, sizes); + } + + /** + * Splits the source array into multiple array segments using this separator, + * as many times as splits are found. This will naturally produce copied byte + * arrays for each of the split segments. + * + * @param source byte array to be split + * @return source split by this separator. + */ + public byte[][] split(byte[] source) { + return split(source, this.bytes); + } + + /** + * Returns a list of ranges identifying [start, end) -- closed, open -- + * positions within the source byte array that would be split using the + * separator byte array. + * The sizes indicate the sizes of the relative components/segments. + * In case one of the segments contains this separator before the specified + * size is reached, the separator will be considered part of that segment and + * we will continue till size is reached. + * Variable length strings cannot contain this separator and are indiced with + * a size of {@value #VARIABLE_SIZE}. Such strings are encoded for this + * separator and decoded after the results from split is returned. + * + * @param source the source data + * @param separator the separator pattern to look for + * @param sizes indicate the sizes of the relative components/segments. + * @return a list of ranges. + */ + private static List<Range> splitRanges(byte[] source, byte[] separator, + int[] sizes) { + List<Range> segments = new ArrayList<Range>(); + if (source == null || separator == null) { + return segments; + } + // VARIABLE_SIZE here indicates that there is no limit to number of segments + // to return. + int limit = VARIABLE_SIZE; + if (sizes != null && sizes.length > 0) { + limit = sizes.length; + } + int start = 0; + int currentSegment = 0; + itersource: for (int i = 0; i < source.length; i++) { + for (int j = 0; j < separator.length; j++) { + if (source[i + j] != separator[j]) { + continue itersource; + } + } + // all separator elements matched + if (limit > VARIABLE_SIZE) { + if (segments.size() >= (limit - 1)) { + // everything else goes in one final segment + break; + } + if (sizes != null) { + int currentSegExpectedSize = sizes[currentSegment]; + if (currentSegExpectedSize > VARIABLE_SIZE) { + int currentSegSize = i - start; + if (currentSegSize < currentSegExpectedSize) { + // Segment not yet complete. More bytes to parse. + continue itersource; + } else if (currentSegSize > currentSegExpectedSize) { + // Segment is not as per size. + throw new IllegalArgumentException( + "Segments not separated as per expected sizes"); + } + } + } + } + segments.add(new Range(start, i)); + start = i + separator.length; + // i will be incremented again in outer for loop + i += separator.length - 1; + currentSegment++; + } + // add in remaining to a final range + if (start <= source.length) { + if (sizes != null) { + // Check if final segment is as per size specified. + if (sizes[currentSegment] > VARIABLE_SIZE && + source.length - start > sizes[currentSegment]) { + // Segment is not as per size. + throw new IllegalArgumentException( + "Segments not separated as per expected sizes"); + } + } + segments.add(new Range(start, source.length)); + } + return segments; + } + + /** + * Splits based on segments calculated based on limit/sizes specified for the + * separator. + * + * @param source byte array to be split. + * @param segments specifies the range for each segment. + * @return a byte[][] split as per the segment ranges. + */ + private static byte[][] split(byte[] source, List<Range> segments) { + byte[][] splits = new byte[segments.size()][]; + for (int i = 0; i < segments.size(); i++) { + Range r = segments.get(i); + byte[] tmp = new byte[r.length()]; + if (tmp.length > 0) { + System.arraycopy(source, r.start(), tmp, 0, r.length()); + } + splits[i] = tmp; + } + return splits; + } + + /** + * Splits the source array into multiple array segments using the given + * separator based on the sizes. This will naturally produce copied byte + * arrays for each of the split segments. + * + * @param source source array. + * @param separator separator represented as a byte array. + * @param sizes sizes of relative components/segments. + * @return byte[][] after splitting the source. + */ + private static byte[][] split(byte[] source, byte[] separator, int[] sizes) { + List<Range> segments = splitRanges(source, separator, sizes); + return split(source, segments); + } + + /** + * Splits the source array into multiple array segments using the given + * separator. This will naturally produce copied byte arrays for each of the + * split segments. + * + * @param source Source array. + * @param separator Separator represented as a byte array. + * @return byte[][] after splitting the source. + */ + private static byte[][] split(byte[] source, byte[] separator) { + return split(source, separator, (int[]) null); + } + + /** + * Splits the source array into multiple array segments using the given + * separator, up to a maximum of count items. This will naturally produce + * copied byte arrays for each of the split segments. + * + * @param source Source array. + * @param separator Separator represented as a byte array. + * @param limit a non-positive value indicates no limit on number of segments. + * @return byte[][] after splitting the input source. + */ + private static byte[][] split(byte[] source, byte[] separator, int limit) { + int[] sizes = null; + if (limit > VARIABLE_SIZE) { + sizes = new int[limit]; + } + List<Range> segments = splitRanges(source, separator, sizes); + return split(source, segments); + } +} http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/StringKeyConverter.java ---------------------------------------------------------------------- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/StringKeyConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/StringKeyConverter.java new file mode 100644 index 0000000..282848e --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/StringKeyConverter.java @@ -0,0 +1,54 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +/** + * Encodes and decodes column names / row keys which are merely strings. + * Column prefixes are not part of the column name passed for encoding. It is + * added later, if required in the associated ColumnPrefix implementations. + */ +public final class StringKeyConverter implements KeyConverter<String> { + + public StringKeyConverter() { + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter + * #encode(java.lang.Object) + */ + @Override + public byte[] encode(String key) { + return Separator.encode(key, Separator.SPACE, Separator.TAB); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter + * #decode(byte[]) + */ + @Override + public String decode(byte[] bytes) { + return Separator.decode(bytes, Separator.TAB, Separator.SPACE); + } +} http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimestampGenerator.java ---------------------------------------------------------------------- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimestampGenerator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimestampGenerator.java new file mode 100644 index 0000000..d03b37d --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TimestampGenerator.java @@ -0,0 +1,116 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * 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.util.concurrent.atomic.AtomicLong; + +import org.apache.hadoop.yarn.api.records.ApplicationId; + +/** + * Utility class that allows HBase coprocessors to interact with unique + * timestamps. + */ +public class TimestampGenerator { + + /* + * if this is changed, then reading cell timestamps written with older + * multiplier value will not work + */ + public static final long TS_MULTIPLIER = 1000000L; + + private final AtomicLong lastTimestamp = new AtomicLong(); + + /** + * Returns the current wall clock time in milliseconds, multiplied by the + * required precision. + * + * @return current timestamp. + */ + public long currentTime() { + // We want to align cell timestamps with current time. + // cell timestamps are not be less than + // System.currentTimeMillis() * TS_MULTIPLIER. + return System.currentTimeMillis() * TS_MULTIPLIER; + } + + /** + * Returns a timestamp value unique within the scope of this + * {@code TimestampGenerator} instance. For usage by HBase + * {@code RegionObserver} coprocessors, this normally means unique within a + * given region. + * + * Unlikely scenario of generating a non-unique timestamp: if there is a + * sustained rate of more than 1M hbase writes per second AND if region fails + * over within that time range of timestamps being generated then there may be + * collisions writing to a cell version of the same column. + * + * @return unique timestamp. + */ + public long getUniqueTimestamp() { + long lastTs; + long nextTs; + do { + lastTs = lastTimestamp.get(); + nextTs = Math.max(lastTs + 1, currentTime()); + } while (!lastTimestamp.compareAndSet(lastTs, nextTs)); + return nextTs; + } + + /** + * Returns a timestamp multiplied with TS_MULTIPLIER and last few digits of + * application id. + * + * Unlikely scenario of generating a timestamp that is a duplicate: If more + * than a 1M concurrent apps are running in one flow run AND write to same + * column at the same time, then say appId of 1M and 1 will overlap + * with appId of 001 and there may be collisions for that flow run's + * specific column. + * + * @param incomingTS Timestamp to be converted. + * @param appId Application Id. + * @return a timestamp multiplied with TS_MULTIPLIER and last few digits of + * application id + */ + public static long getSupplementedTimestamp(long incomingTS, String appId) { + long suffix = getAppIdSuffix(appId); + long outgoingTS = incomingTS * TS_MULTIPLIER + suffix; + return outgoingTS; + + } + + private static long getAppIdSuffix(String appIdStr) { + if (appIdStr == null) { + return 0L; + } + ApplicationId appId = ApplicationId.fromString(appIdStr); + long id = appId.getId() % TS_MULTIPLIER; + return id; + } + + /** + * truncates the last few digits of the timestamp which were supplemented by + * the TimestampGenerator#getSupplementedTimestamp function. + * + * @param incomingTS Timestamp to be truncated. + * @return a truncated timestamp value + */ + public static long getTruncatedTimestamp(long incomingTS) { + return incomingTS / TS_MULTIPLIER; + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ValueConverter.java ---------------------------------------------------------------------- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ValueConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ValueConverter.java new file mode 100644 index 0000000..757a6d3 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/ValueConverter.java @@ -0,0 +1,47 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * 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; + +/** + * Converter used to encode/decode value associated with a column prefix or a + * column. + */ +public interface ValueConverter { + + /** + * Encode an object as a byte array depending on the converter implementation. + * + * @param value Value to be encoded. + * @return a byte array + * @throws IOException if any problem is encountered while encoding. + */ + byte[] encodeValue(Object value) throws IOException; + + /** + * Decode a byte array and convert it into an object depending on the + * converter implementation. + * + * @param bytes Byte array to be decoded. + * @return an object + * @throws IOException if any problem is encountered while decoding. + */ + Object decodeValue(byte[] bytes) throws IOException; +} http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/package-info.java ---------------------------------------------------------------------- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/package-info.java new file mode 100644 index 0000000..0df5b8a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/package-info.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Package org.apache.hadoop.yarn.server.timelineservice.storage.common contains + * a set of utility classes used across backend storage reader and writer. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.yarn.server.timelineservice.storage.common; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumn.java ---------------------------------------------------------------------- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumn.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumn.java new file mode 100644 index 0000000..81961d3 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumn.java @@ -0,0 +1,105 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.entity; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Column; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; + +/** + * Identifies fully qualified columns for the {@link EntityTable}. + */ +public enum EntityColumn implements Column<EntityTable> { + + /** + * Identifier for the entity. + */ + ID(EntityColumnFamily.INFO, "id"), + + /** + * The type of entity. + */ + TYPE(EntityColumnFamily.INFO, "type"), + + /** + * When the entity was created. + */ + CREATED_TIME(EntityColumnFamily.INFO, "created_time", new LongConverter()), + + /** + * The version of the flow that this entity belongs to. + */ + FLOW_VERSION(EntityColumnFamily.INFO, "flow_version"); + + private final ColumnFamily<EntityTable> columnFamily; + private final String columnQualifier; + private final byte[] columnQualifierBytes; + private final ValueConverter valueConverter; + + EntityColumn(ColumnFamily<EntityTable> columnFamily, + String columnQualifier) { + this(columnFamily, columnQualifier, GenericConverter.getInstance()); + } + + EntityColumn(ColumnFamily<EntityTable> columnFamily, + String columnQualifier, ValueConverter converter) { + this.columnFamily = columnFamily; + this.columnQualifier = columnQualifier; + // Future-proof by ensuring the right column prefix hygiene. + this.columnQualifierBytes = + Bytes.toBytes(Separator.SPACE.encode(columnQualifier)); + this.valueConverter = converter; + } + + /** + * @return the column name value + */ + private String getColumnQualifier() { + return columnQualifier; + } + + @Override + public byte[] getColumnQualifierBytes() { + return columnQualifierBytes.clone(); + } + + @Override + public byte[] getColumnFamilyBytes() { + return columnFamily.getBytes(); + } + + @Override + public ValueConverter getValueConverter() { + return valueConverter; + } + + @Override + public Attribute[] getCombinedAttrsWithAggr(Attribute... attributes) { + return attributes; + } + + @Override + public boolean supplementCellTimestamp() { + return false; + } +} http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnFamily.java ---------------------------------------------------------------------- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnFamily.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnFamily.java new file mode 100644 index 0000000..7c63727 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnFamily.java @@ -0,0 +1,65 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.entity; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; + +/** + * Represents the entity table column families. + */ +public enum EntityColumnFamily implements ColumnFamily<EntityTable> { + + /** + * Info column family houses known columns, specifically ones included in + * columnfamily filters. + */ + INFO("i"), + + /** + * Configurations are in a separate column family for two reasons: a) the size + * of the config values can be very large and b) we expect that config values + * are often separately accessed from other metrics and info columns. + */ + CONFIGS("c"), + + /** + * Metrics have a separate column family, because they have a separate TTL. + */ + METRICS("m"); + + /** + * Byte representation of this column family. + */ + private final byte[] bytes; + + /** + * @param value create a column family with this name. Must be lower case and + * without spaces. + */ + EntityColumnFamily(String value) { + // column families should be lower case and not contain any spaces. + this.bytes = Bytes.toBytes(Separator.SPACE.encode(value)); + } + + public byte[] getBytes() { + return Bytes.copy(bytes); + } + +} http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java ---------------------------------------------------------------------- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java new file mode 100644 index 0000000..08234d3 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityColumnPrefix.java @@ -0,0 +1,162 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.entity; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnFamily; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnHelper; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ColumnPrefix; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.GenericConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.ValueConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute; + +/** + * Identifies partially qualified columns for the entity table. + */ +public enum EntityColumnPrefix implements ColumnPrefix<EntityTable> { + + /** + * To store TimelineEntity getIsRelatedToEntities values. + */ + IS_RELATED_TO(EntityColumnFamily.INFO, "s"), + + /** + * To store TimelineEntity getRelatesToEntities values. + */ + RELATES_TO(EntityColumnFamily.INFO, "r"), + + /** + * To store TimelineEntity info values. + */ + INFO(EntityColumnFamily.INFO, "i"), + + /** + * Lifecycle events for an entity. + */ + EVENT(EntityColumnFamily.INFO, "e", true), + + /** + * Config column stores configuration with config key as the column name. + */ + CONFIG(EntityColumnFamily.CONFIGS, null), + + /** + * Metrics are stored with the metric name as the column name. + */ + METRIC(EntityColumnFamily.METRICS, null, new LongConverter()); + + private final ColumnFamily<EntityTable> columnFamily; + + /** + * Can be null for those cases where the provided column qualifier is the + * entire column name. + */ + private final String columnPrefix; + private final byte[] columnPrefixBytes; + private final ValueConverter valueConverter; + + /** + * Private constructor, meant to be used by the enum definition. + * + * @param columnFamily that this column is stored in. + * @param columnPrefix for this column. + */ + EntityColumnPrefix(ColumnFamily<EntityTable> columnFamily, + String columnPrefix) { + this(columnFamily, columnPrefix, false, GenericConverter.getInstance()); + } + + EntityColumnPrefix(ColumnFamily<EntityTable> columnFamily, + String columnPrefix, boolean compondColQual) { + this(columnFamily, columnPrefix, compondColQual, + GenericConverter.getInstance()); + } + + EntityColumnPrefix(ColumnFamily<EntityTable> columnFamily, + String columnPrefix, ValueConverter converter) { + this(columnFamily, columnPrefix, false, converter); + } + + /** + * Private constructor, meant to be used by the enum definition. + * + * @param columnFamily that this column is stored in. + * @param columnPrefix for this column. + * @param converter used to encode/decode values to be stored in HBase for + * this column prefix. + */ + EntityColumnPrefix(ColumnFamily<EntityTable> columnFamily, + String columnPrefix, boolean compondColQual, ValueConverter converter) { + this.valueConverter = converter; + this.columnFamily = columnFamily; + this.columnPrefix = columnPrefix; + if (columnPrefix == null) { + this.columnPrefixBytes = null; + } else { + // Future-proof by ensuring the right column prefix hygiene. + this.columnPrefixBytes = + Bytes.toBytes(Separator.SPACE.encode(columnPrefix)); + } + } + + /** + * @return the column name value + */ + public String getColumnPrefix() { + return columnPrefix; + } + + @Override + public byte[] getColumnPrefixBytes(byte[] qualifierPrefix) { + return ColumnHelper.getColumnQualifier( + this.columnPrefixBytes, qualifierPrefix); + } + + @Override + public byte[] getColumnPrefixBytes(String qualifierPrefix) { + return ColumnHelper.getColumnQualifier( + this.columnPrefixBytes, qualifierPrefix); + } + + @Override + public byte[] getColumnPrefixInBytes() { + return columnPrefixBytes != null ? columnPrefixBytes.clone() : null; + } + + @Override + public byte[] getColumnFamilyBytes() { + return columnFamily.getBytes(); + } + + @Override + public ValueConverter getValueConverter() { + return valueConverter; + } + + @Override + public Attribute[] getCombinedAttrsWithAggr(Attribute... attributes) { + return attributes; + } + + @Override + public boolean supplementCellTimeStamp() { + return false; + } +} http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKey.java ---------------------------------------------------------------------- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKey.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKey.java new file mode 100644 index 0000000..b85a9b0 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKey.java @@ -0,0 +1,299 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.entity; + +import java.util.List; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.yarn.server.timelineservice.reader.TimelineReaderUtils; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.AppIdKeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverterToString; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.LongConverter; +import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator; + +/** + * Represents a rowkey for the entity table. + */ +public class EntityRowKey { + private final String clusterId; + private final String userId; + private final String flowName; + private final Long flowRunId; + private final String appId; + private final String entityType; + private final Long entityIdPrefix; + private final String entityId; + private final EntityRowKeyConverter entityRowKeyConverter = + new EntityRowKeyConverter(); + + public EntityRowKey(String clusterId, String userId, String flowName, + Long flowRunId, String appId, String entityType, Long entityIdPrefix, + String entityId) { + this.clusterId = clusterId; + this.userId = userId; + this.flowName = flowName; + this.flowRunId = flowRunId; + this.appId = appId; + this.entityType = entityType; + this.entityIdPrefix = entityIdPrefix; + this.entityId = entityId; + } + + public String getClusterId() { + return clusterId; + } + + public String getUserId() { + return userId; + } + + public String getFlowName() { + return flowName; + } + + public Long getFlowRunId() { + return flowRunId; + } + + public String getAppId() { + return appId; + } + + public String getEntityType() { + return entityType; + } + + public String getEntityId() { + return entityId; + } + + public Long getEntityIdPrefix() { + return entityIdPrefix; + } + + /** + * Constructs a row key for the entity table as follows: + * {@code userName!clusterId!flowName!flowRunId!AppId!entityType!entityId}. + * Typically used while querying a specific entity. + * + * @return byte array with the row key. + */ + public byte[] getRowKey() { + return entityRowKeyConverter.encode(this); + } + + /** + * Given the raw row key as bytes, returns the row key as an object. + * @param rowKey byte representation of row key. + * @return An <cite>EntityRowKey</cite> object. + */ + public static EntityRowKey parseRowKey(byte[] rowKey) { + return new EntityRowKeyConverter().decode(rowKey); + } + + /** + * Constructs a row key for the entity table as follows: + * <p> + * {@code userName!clusterId!flowName!flowRunId!AppId! + * entityType!entityIdPrefix!entityId}. + * </p> + * @return String representation of row key. + */ + public String getRowKeyAsString() { + return entityRowKeyConverter.encodeAsString(this); + } + + /** + * Given the encoded row key as string, returns the row key as an object. + * @param encodedRowKey String representation of row key. + * @return A <cite>EntityRowKey</cite> object. + */ + public static EntityRowKey parseRowKeyFromString(String encodedRowKey) { + return new EntityRowKeyConverter().decodeFromString(encodedRowKey); + } + + /** + * Encodes and decodes row key for entity table. The row key is of the form : + * userName!clusterId!flowName!flowRunId!appId!entityType!entityId. flowRunId + * is a long, appId is encoded/decoded using {@link AppIdKeyConverter} and + * rest are strings. + * <p> + */ + final private static class EntityRowKeyConverter implements + KeyConverter<EntityRowKey>, KeyConverterToString<EntityRowKey> { + + private final AppIdKeyConverter appIDKeyConverter = new AppIdKeyConverter(); + + private EntityRowKeyConverter() { + } + + /** + * Entity row key is of the form + * userName!clusterId!flowName!flowRunId!appId!entityType!entityId w. each + * segment separated by !. The sizes below indicate sizes of each one of + * these segments in sequence. clusterId, userName, flowName, entityType and + * entityId are strings. flowrunId is a long hence 8 bytes in size. app id + * is represented as 12 bytes with cluster timestamp part of appid being 8 + * bytes (long) and seq id being 4 bytes(int). Strings are variable in size + * (i.e. end whenever separator is encountered). This is used while decoding + * and helps in determining where to split. + */ + private static final int[] SEGMENT_SIZES = {Separator.VARIABLE_SIZE, + Separator.VARIABLE_SIZE, Separator.VARIABLE_SIZE, Bytes.SIZEOF_LONG, + AppIdKeyConverter.getKeySize(), Separator.VARIABLE_SIZE, + Bytes.SIZEOF_LONG, Separator.VARIABLE_SIZE }; + + /* + * (non-Javadoc) + * + * Encodes EntityRowKey object into a byte array with each component/field + * in EntityRowKey separated by Separator#QUALIFIERS. This leads to an + * entity table row key of the form + * userName!clusterId!flowName!flowRunId!appId!entityType!entityId If + * entityType in passed EntityRowKey object is null (and the fields + * preceding it i.e. clusterId, userId and flowName, flowRunId and appId + * are not null), this returns a row key prefix of the form + * userName!clusterId!flowName!flowRunId!appId! and if entityId in + * EntityRowKey is null (other 6 components are not null), this returns a + * row key prefix of the form + * userName!clusterId!flowName!flowRunId!appId!entityType! flowRunId is + * inverted while encoding as it helps maintain a descending order for row + * keys in entity table. + * + * @see org.apache.hadoop.yarn.server.timelineservice.storage.common + * .KeyConverter#encode(java.lang.Object) + */ + @Override + public byte[] encode(EntityRowKey rowKey) { + byte[] user = + Separator.encode(rowKey.getUserId(), Separator.SPACE, Separator.TAB, + Separator.QUALIFIERS); + byte[] cluster = + Separator.encode(rowKey.getClusterId(), Separator.SPACE, + Separator.TAB, Separator.QUALIFIERS); + byte[] flow = + Separator.encode(rowKey.getFlowName(), Separator.SPACE, + Separator.TAB, Separator.QUALIFIERS); + byte[] first = Separator.QUALIFIERS.join(user, cluster, flow); + // Note that flowRunId is a long, so we can't encode them all at the same + // time. + byte[] second = + Bytes.toBytes(LongConverter.invertLong(rowKey.getFlowRunId())); + byte[] third = appIDKeyConverter.encode(rowKey.getAppId()); + if (rowKey.getEntityType() == null) { + return Separator.QUALIFIERS.join(first, second, third, + Separator.EMPTY_BYTES); + } + byte[] entityType = + Separator.encode(rowKey.getEntityType(), Separator.SPACE, + Separator.TAB, Separator.QUALIFIERS); + + if (rowKey.getEntityIdPrefix() == null) { + return Separator.QUALIFIERS.join(first, second, third, entityType, + Separator.EMPTY_BYTES); + } + + byte[] entityIdPrefix = Bytes.toBytes(rowKey.getEntityIdPrefix()); + + if (rowKey.getEntityId() == null) { + return Separator.QUALIFIERS.join(first, second, third, entityType, + entityIdPrefix, Separator.EMPTY_BYTES); + } + + byte[] entityId = Separator.encode(rowKey.getEntityId(), Separator.SPACE, + Separator.TAB, Separator.QUALIFIERS); + + byte[] fourth = + Separator.QUALIFIERS.join(entityType, entityIdPrefix, entityId); + + return Separator.QUALIFIERS.join(first, second, third, fourth); + } + + /* + * (non-Javadoc) + * + * Decodes an application row key of the form + * userName!clusterId!flowName!flowRunId!appId!entityType!entityId + * represented in byte format and converts it into an EntityRowKey object. + * flowRunId is inverted while decoding as it was inverted while encoding. + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.common + * .KeyConverter#decode(byte[]) + */ + @Override + public EntityRowKey decode(byte[] rowKey) { + byte[][] rowKeyComponents = + Separator.QUALIFIERS.split(rowKey, SEGMENT_SIZES); + if (rowKeyComponents.length != 8) { + throw new IllegalArgumentException("the row key is not valid for " + + "an entity"); + } + String userId = + Separator.decode(Bytes.toString(rowKeyComponents[0]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + String clusterId = + Separator.decode(Bytes.toString(rowKeyComponents[1]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + String flowName = + Separator.decode(Bytes.toString(rowKeyComponents[2]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + Long flowRunId = + LongConverter.invertLong(Bytes.toLong(rowKeyComponents[3])); + String appId = appIDKeyConverter.decode(rowKeyComponents[4]); + String entityType = + Separator.decode(Bytes.toString(rowKeyComponents[5]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + + Long entityPrefixId = Bytes.toLong(rowKeyComponents[6]); + + String entityId = + Separator.decode(Bytes.toString(rowKeyComponents[7]), + Separator.QUALIFIERS, Separator.TAB, Separator.SPACE); + return new EntityRowKey(clusterId, userId, flowName, flowRunId, appId, + entityType, entityPrefixId, entityId); + } + + @Override + public String encodeAsString(EntityRowKey key) { + if (key.clusterId == null || key.userId == null || key.flowName == null + || key.flowRunId == null || key.appId == null + || key.entityType == null || key.entityIdPrefix == null + || key.entityId == null) { + throw new IllegalArgumentException(); + } + return TimelineReaderUtils + .joinAndEscapeStrings(new String[] {key.clusterId, key.userId, + key.flowName, key.flowRunId.toString(), key.appId, key.entityType, + key.entityIdPrefix.toString(), key.entityId}); + } + + @Override + public EntityRowKey decodeFromString(String encodedRowKey) { + List<String> split = TimelineReaderUtils.split(encodedRowKey); + if (split == null || split.size() != 8) { + throw new IllegalArgumentException("Invalid row key for entity table."); + } + Long flowRunId = Long.valueOf(split.get(3)); + Long entityIdPrefix = Long.valueOf(split.get(6)); + return new EntityRowKey(split.get(0), split.get(1), split.get(2), + flowRunId, split.get(4), split.get(5), entityIdPrefix, split.get(7)); + } + } +} http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKeyPrefix.java ---------------------------------------------------------------------- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKeyPrefix.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKeyPrefix.java new file mode 100644 index 0000000..47a1789 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityRowKeyPrefix.java @@ -0,0 +1,77 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.yarn.server.timelineservice.storage.entity; + +import org.apache.hadoop.yarn.server.timelineservice.storage.common.RowKeyPrefix; + +/** + * Represents a partial rowkey without the entityId or without entityType and + * entityId for the entity table. + * + */ +public class EntityRowKeyPrefix extends EntityRowKey implements + RowKeyPrefix<EntityRowKey> { + + /** + * Creates a prefix which generates the following rowKeyPrefixes for the + * entity table: + * {@code userName!clusterId!flowName!flowRunId!AppId!entityType!}. + * @param clusterId identifying the cluster + * @param userId identifying the user + * @param flowName identifying the flow + * @param flowRunId identifying the individual run of this flow + * @param appId identifying the application + * @param entityType which entity type + * @param entityIdPrefix for entityId + * @param entityId for an entity + */ + public EntityRowKeyPrefix(String clusterId, String userId, String flowName, + Long flowRunId, String appId, String entityType, Long entityIdPrefix, + String entityId) { + super(clusterId, userId, flowName, flowRunId, appId, entityType, + entityIdPrefix, entityId); + } + + /** + * Creates a prefix which generates the following rowKeyPrefixes for the + * entity table: + * {@code userName!clusterId!flowName!flowRunId!AppId!entityType!entityId}. + * + * @param clusterId identifying the cluster + * @param userId identifying the user + * @param flowName identifying the flow + * @param flowRunId identifying the individual run of this flow + * @param appId identifying the application + */ + public EntityRowKeyPrefix(String clusterId, String userId, String flowName, + Long flowRunId, String appId) { + this(clusterId, userId, flowName, flowRunId, appId, null, null, null); + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.yarn.server.timelineservice.storage.application. + * RowKeyPrefix#getRowKeyPrefix() + */ + public byte[] getRowKeyPrefix() { + return super.getRowKey(); + } + +} http://git-wip-us.apache.org/repos/asf/hadoop/blob/9af30d46/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java ---------------------------------------------------------------------- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java new file mode 100644 index 0000000..dceeb99 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/entity/EntityTable.java @@ -0,0 +1,61 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.timelineservice.storage.entity; + +import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTable; + +/** + * The entity table as column families info, config and metrics. Info stores + * information about a timeline entity object config stores configuration data + * of a timeline entity object metrics stores the metrics of a timeline entity + * object + * + * Example entity table record: + * + * <pre> + * |-------------------------------------------------------------------------| + * | Row | Column Family | Column Family| Column Family| + * | key | info | metrics | config | + * |-------------------------------------------------------------------------| + * | userName! | id:entityId | metricId1: | configKey1: | + * | clusterId! | | metricValue1 | configValue1 | + * | flowName! | type:entityType | @timestamp1 | | + * | flowRunId! | | | configKey2: | + * | AppId! | created_time: | metricId1: | configValue2 | + * | entityType!| 1392993084018 | metricValue2 | | + * | idPrefix! | | @timestamp2 | | + * | entityId | i!infoKey: | | | + * | | infoValue | metricId1: | | + * | | | metricValue1 | | + * | | r!relatesToKey: | @timestamp2 | | + * | | id3=id4=id5 | | | + * | | | | | + * | | s!isRelatedToKey | | | + * | | id7=id9=id6 | | | + * | | | | | + * | | e!eventId=timestamp=infoKey: | | | + * | | eventInfoValue | | | + * | | | | | + * | | flowVersion: | | | + * | | versionValue | | | + * |-------------------------------------------------------------------------| + * </pre> + */ +public final class EntityTable extends BaseTable<EntityTable> { +} --------------------------------------------------------------------- To unsubscribe, e-mail: common-commits-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-commits-h...@hadoop.apache.org