|
@@ -20,6 +20,12 @@ package org.apache.hadoop.utils.db;
|
|
|
|
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
|
|
|
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
|
|
+import org.apache.hadoop.utils.db.cache.CacheKey;
|
|
|
|
+import org.apache.hadoop.utils.db.cache.CacheValue;
|
|
|
|
+import org.apache.hadoop.utils.db.cache.PartialTableCache;
|
|
|
|
+import org.apache.hadoop.utils.db.cache.TableCache;
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Strongly typed table implementation.
|
|
* Strongly typed table implementation.
|
|
* <p>
|
|
* <p>
|
|
@@ -31,13 +37,16 @@ import java.io.IOException;
|
|
*/
|
|
*/
|
|
public class TypedTable<KEY, VALUE> implements Table<KEY, VALUE> {
|
|
public class TypedTable<KEY, VALUE> implements Table<KEY, VALUE> {
|
|
|
|
|
|
- private Table<byte[], byte[]> rawTable;
|
|
|
|
|
|
+ private final Table<byte[], byte[]> rawTable;
|
|
|
|
+
|
|
|
|
+ private final CodecRegistry codecRegistry;
|
|
|
|
|
|
- private CodecRegistry codecRegistry;
|
|
|
|
|
|
+ private final Class<KEY> keyType;
|
|
|
|
|
|
- private Class<KEY> keyType;
|
|
|
|
|
|
+ private final Class<VALUE> valueType;
|
|
|
|
+
|
|
|
|
+ private final TableCache<CacheKey<KEY>, CacheValue<VALUE>> cache;
|
|
|
|
|
|
- private Class<VALUE> valueType;
|
|
|
|
|
|
|
|
public TypedTable(
|
|
public TypedTable(
|
|
Table<byte[], byte[]> rawTable,
|
|
Table<byte[], byte[]> rawTable,
|
|
@@ -47,6 +56,7 @@ public class TypedTable<KEY, VALUE> implements Table<KEY, VALUE> {
|
|
this.codecRegistry = codecRegistry;
|
|
this.codecRegistry = codecRegistry;
|
|
this.keyType = keyType;
|
|
this.keyType = keyType;
|
|
this.valueType = valueType;
|
|
this.valueType = valueType;
|
|
|
|
+ cache = new PartialTableCache<>();
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -69,8 +79,34 @@ public class TypedTable<KEY, VALUE> implements Table<KEY, VALUE> {
|
|
return rawTable.isEmpty();
|
|
return rawTable.isEmpty();
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Returns the value mapped to the given key in byte array or returns null
|
|
|
|
+ * if the key is not found.
|
|
|
|
+ *
|
|
|
|
+ * Caller's of this method should use synchronization mechanism, when
|
|
|
|
+ * accessing. First it will check from cache, if it has entry return the
|
|
|
|
+ * value, otherwise get from the RocksDB table.
|
|
|
|
+ *
|
|
|
|
+ * @param key metadata key
|
|
|
|
+ * @return VALUE
|
|
|
|
+ * @throws IOException
|
|
|
|
+ */
|
|
@Override
|
|
@Override
|
|
public VALUE get(KEY key) throws IOException {
|
|
public VALUE get(KEY key) throws IOException {
|
|
|
|
+ // Here the metadata lock will guarantee that cache is not updated for same
|
|
|
|
+ // key during get key.
|
|
|
|
+ CacheValue< VALUE > cacheValue = cache.get(new CacheKey<>(key));
|
|
|
|
+ if (cacheValue == null) {
|
|
|
|
+ // If no cache for the table or if it does not exist in cache get from
|
|
|
|
+ // RocksDB table.
|
|
|
|
+ return getFromTable(key);
|
|
|
|
+ } else {
|
|
|
|
+ // We have a value in cache, return the value.
|
|
|
|
+ return cacheValue.getValue();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private VALUE getFromTable(KEY key) throws IOException {
|
|
byte[] keyBytes = codecRegistry.asRawData(key);
|
|
byte[] keyBytes = codecRegistry.asRawData(key);
|
|
byte[] valueBytes = rawTable.get(keyBytes);
|
|
byte[] valueBytes = rawTable.get(keyBytes);
|
|
return codecRegistry.asObject(valueBytes, valueType);
|
|
return codecRegistry.asObject(valueBytes, valueType);
|
|
@@ -106,6 +142,40 @@ public class TypedTable<KEY, VALUE> implements Table<KEY, VALUE> {
|
|
|
|
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @Override
|
|
|
|
+ public void addCacheEntry(CacheKey<KEY> cacheKey,
|
|
|
|
+ CacheValue<VALUE> cacheValue) {
|
|
|
|
+ // This will override the entry if there is already entry for this key.
|
|
|
|
+ cache.put(cacheKey, cacheValue);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public void cleanupCache(long epoch) {
|
|
|
|
+ cache.cleanup(epoch);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @VisibleForTesting
|
|
|
|
+ TableCache<CacheKey<KEY>, CacheValue<VALUE>> getCache() {
|
|
|
|
+ return cache;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public Table<byte[], byte[]> getRawTable() {
|
|
|
|
+ return rawTable;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public CodecRegistry getCodecRegistry() {
|
|
|
|
+ return codecRegistry;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public Class<KEY> getKeyType() {
|
|
|
|
+ return keyType;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public Class<VALUE> getValueType() {
|
|
|
|
+ return valueType;
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Key value implementation for strongly typed tables.
|
|
* Key value implementation for strongly typed tables.
|
|
*/
|
|
*/
|