parquet-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jul...@apache.org
Subject git commit: PARQUET-4: Use LRU caching for footers in ParquetInputFormat.
Date Fri, 18 Jul 2014 23:02:22 GMT
Repository: incubator-parquet-mr
Updated Branches:
  refs/heads/master 2d8ebdbe0 -> 5dffe3521


PARQUET-4: Use LRU caching for footers in ParquetInputFormat.

Reopening https://github.com/Parquet/parquet-mr/pull/403 against the new Apache repository.

Author: Matthieu Martin <ma.tt.b.ma.rt.in+parquet@gmail.com>

Closes #2 from matt-martin/master and squashes the following commits:

99bb5a3 [Matthieu Martin] Minor javadoc and whitespace changes. Also added the FileStatusWrapper
class to ParquetInputFormat to make sure that the debugging log statements print out meaningful
paths.
250a398 [Matthieu Martin] Be less aggressive about checking whether the underlying file has
been appended to/overwritten/deleted in order to minimize the number of namenode interactions.
d946445 [Matthieu Martin] Add javadocs to parquet.hadoop.LruCache.  Rename cache "entries"
as cache "values" to avoid confusion with java.util.Map.Entry (which contains key value pairs
whereas our old "entries" really only refer to the values).
a363622 [Matthieu Martin] Use LRU caching for footers in ParquetInputFormat.


Project: http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/commit/5dffe352
Tree: http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/tree/5dffe352
Diff: http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/diff/5dffe352

Branch: refs/heads/master
Commit: 5dffe3521588016cf3519792953b0879054c3bfd
Parents: 2d8ebdb
Author: Matthieu Martin <ma.tt.b.ma.rt.in+parquet@gmail.com>
Authored: Fri Jul 18 16:02:09 2014 -0700
Committer: julien <julien@twitter.com>
Committed: Fri Jul 18 16:02:09 2014 -0700

----------------------------------------------------------------------
 parquet-hadoop/pom.xml                          |   6 +
 .../src/main/java/parquet/hadoop/LruCache.java  | 181 +++++++++++++++++++
 .../java/parquet/hadoop/ParquetInputFormat.java | 143 ++++++++++++++-
 .../java/parquet/hadoop/TestInputFormat.java    |  52 +++++-
 .../test/java/parquet/hadoop/TestLruCache.java  | 144 +++++++++++++++
 5 files changed, 516 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/5dffe352/parquet-hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/parquet-hadoop/pom.xml b/parquet-hadoop/pom.xml
index ec7b94d..c1b4f07 100644
--- a/parquet-hadoop/pom.xml
+++ b/parquet-hadoop/pom.xml
@@ -68,6 +68,12 @@
       <type>jar</type>
       <scope>compile</scope>
     </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <version>1.9.5</version>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/5dffe352/parquet-hadoop/src/main/java/parquet/hadoop/LruCache.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/LruCache.java b/parquet-hadoop/src/main/java/parquet/hadoop/LruCache.java
new file mode 100644
index 0000000..e9ecb37
--- /dev/null
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/LruCache.java
@@ -0,0 +1,181 @@
+package parquet.hadoop;
+
+import parquet.Log;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * A basic implementation of an LRU cache.  Besides evicting the least recently
+ * used entries (either based on insertion or access order), this class also
+ * checks for "stale" entries as entries are inserted or retrieved (note
+ * "staleness" is defined by the entries themselves (see
+ * {@link parquet.hadoop.LruCache.Value}).
+ *
+ * @param <K> The key type. Acts as the key in a {@link java.util.LinkedHashMap}
+ * @param <V> The value type.  Must extend {@link parquet.hadoop.LruCache.Value}
+ *           so that the "staleness" of the value can be easily determined.
+ */
+final class LruCache<K, V extends LruCache.Value<K, V>> {
+  private static final Log LOG = Log.getLog(LruCache.class);
+
+  private static final float DEFAULT_LOAD_FACTOR = 0.75f;
+
+  private final LinkedHashMap<K, V> cacheMap;
+
+  /**
+   * Constructs an access-order based LRU cache with {@code maxSize} entries.
+   * @param maxSize The maximum number of entries to store in the cache.
+   */
+  public LruCache(final int maxSize) {
+    this(maxSize, DEFAULT_LOAD_FACTOR, true);
+  }
+
+  /**
+   * Constructs an LRU cache.
+   *
+   * @param maxSize The maximum number of entries to store in the cache.
+   * @param loadFactor Used to determine the initial capacity.
+   * @param accessOrder the ordering mode - {@code true} for access-order,
+   * {@code false} for insertion-order
+   */
+  public LruCache(final int maxSize, final float loadFactor, final boolean accessOrder) {
+    int initialCapacity = Math.round(maxSize / loadFactor);
+    cacheMap =
+            new LinkedHashMap<K, V>(initialCapacity, loadFactor, accessOrder) {
+              @Override
+              public boolean removeEldestEntry(final Map.Entry<K, V> eldest) {
+                boolean result = size() > maxSize;
+                if (result) {
+                  if (Log.DEBUG) {
+                    LOG.debug("Removing eldest entry in cache: "
+                            + eldest.getKey());
+                  }
+                }
+                return result;
+              }
+            };
+  }
+
+  /**
+   * Removes the mapping for the specified key from this cache if present.
+   * @param key key whose mapping is to be removed from the cache
+   * @return the previous value associated with key, or null if there was no
+   * mapping for key.
+   */
+  public V remove(final K key) {
+    V oldValue = cacheMap.remove(key);
+    if (oldValue != null) {
+      if (Log.DEBUG) {
+        LOG.debug("Removed cache entry for '" + key + "'");
+      }
+    }
+    return oldValue;
+  }
+
+  /**
+   * Associates the specified value with the specified key in this cache. The
+   * value is only inserted if it is not null and it is considered current. If
+   * the cache previously contained a mapping for the key, the old value is
+   * replaced only if the new value is "newer" than the old one.
+   * @param key key with which the specified value is to be associated
+   * @param newValue value to be associated with the specified key
+   */
+  public void put(final K key, final V newValue) {
+    if (newValue == null || !newValue.isCurrent(key)) {
+      if (Log.WARN) {
+        LOG.warn("Ignoring new cache entry for '" + key + "' because it is "
+                + (newValue == null ? "null" : "not current"));
+      }
+      return;
+    }
+
+    V oldValue = cacheMap.get(key);
+    if (oldValue != null && oldValue.isNewerThan(newValue)) {
+      if (Log.WARN) {
+        LOG.warn("Ignoring new cache entry for '" + key + "' because "
+                + "existing cache entry is newer");
+      }
+      return;
+    }
+
+    // no existing value or new value is newer than old value
+    oldValue = cacheMap.put(key, newValue);
+    if (Log.DEBUG) {
+      if (oldValue == null) {
+        LOG.debug("Added new cache entry for '" + key + "'");
+      } else {
+        LOG.debug("Overwrote existing cache entry for '" + key + "'");
+      }
+    }
+  }
+
+  /**
+   * Removes all of the mappings from this cache. The cache will be empty
+   * after this call returns.
+   */
+  public void clear() {
+    cacheMap.clear();
+  }
+
+  /**
+   * Returns the value to which the specified key is mapped, or null if 1) the
+   * value is not current or 2) this cache contains no mapping for the key.
+   * @param key the key whose associated value is to be returned
+   * @return the value to which the specified key is mapped, or null if 1) the
+   * value is not current or 2) this cache contains no mapping for the key
+   */
+  public V getCurrentValue(final K key) {
+    V value = cacheMap.get(key);
+    if (Log.DEBUG) {
+      LOG.debug("Value for '" + key + "' " + (value == null ? "not " : "")
+              + "in cache");
+    }
+    if (value != null && !value.isCurrent(key)) {
+      // value is not current; remove it and return null
+      remove(key);
+      return null;
+    }
+
+    return value;
+  }
+
+  /**
+   * Returns the number of key-value mappings in this cache.
+   * @return the number of key-value mappings in this cache.
+   */
+  public int size() {
+    return cacheMap.size();
+  }
+
+  /**
+   * {@link parquet.hadoop.LruCache} expects all values to follow this
+   * interface so the cache can determine 1) whether values are current (e.g.
+   * the referenced data has not been modified/updated in such a way that the
+   * value is no longer useful) and 2) whether a value is strictly "newer"
+   * than another value.
+   *
+   * @param <K> The key type.
+   * @param <V> Provides a bound for the {@link #isNewerThan(V)} method
+   */
+  interface Value<K, V> {
+    /**
+     * Is the value still current (e.g. has the referenced data been
+     * modified/updated in such a way that the value is no longer useful)
+     * @param key the key associated with this value
+     * @return {@code true} the value is still current, {@code false} the value
+     * is no longer useful
+     */
+    boolean isCurrent(K key);
+
+    /**
+     * Compares this value with the specified value to check for relative age.
+     * @param otherValue the value to be compared.
+     * @return {@code true} the value is strictly newer than the other value,
+     * {@code false} the value is older or just
+     * as new as the other value.
+     */
+    boolean isNewerThan(V otherValue);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/5dffe352/parquet-hadoop/src/main/java/parquet/hadoop/ParquetInputFormat.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetInputFormat.java b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetInputFormat.java
index ddf67a7..b09c0ff 100644
--- a/parquet-hadoop/src/main/java/parquet/hadoop/ParquetInputFormat.java
+++ b/parquet-hadoop/src/main/java/parquet/hadoop/ParquetInputFormat.java
@@ -18,9 +18,13 @@ package parquet.hadoop;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
@@ -83,8 +87,11 @@ public class ParquetInputFormat<T> extends FileInputFormat<Void,
T> {
    */
   public static final String STRICT_TYPE_CHECKING = "parquet.strict.typing";
 
+  private static final int MIN_FOOTER_CACHE_SIZE = 100;
+
+  private LruCache<FileStatusWrapper, FootersCacheValue> footersCache;
+
   private Class<?> readSupportClass;
-  private List<Footer> footers;
 
   public static void setReadSupportClass(Job job,  Class<?> readSupportClass) {
     ContextUtil.getConfiguration(job).set(READ_SUPPORT_CLASS, readSupportClass.getName());
@@ -404,9 +411,7 @@ public class ParquetInputFormat<T> extends FileInputFormat<Void,
T> {
   private static List<FileStatus> getAllFileRecursively(
       List<FileStatus> files, Configuration conf) throws IOException {
     List<FileStatus> result = new ArrayList<FileStatus>();
-    int len = files.size();
-    for (int i = 0; i < len; ++i) {
-      FileStatus file = files.get(i);
+    for (FileStatus file : files) {
       if (file.isDir()) {
         Path p = file.getPath();
         FileSystem fs = p.getFileSystem(conf);
@@ -444,10 +449,58 @@ public class ParquetInputFormat<T> extends FileInputFormat<Void,
T> {
    * @throws IOException
    */
   public List<Footer> getFooters(JobContext jobContext) throws IOException {
-    if (footers == null) {
-      footers = getFooters(ContextUtil.getConfiguration(jobContext), listStatus(jobContext));
+    List<FileStatus> statuses = listStatus(jobContext);
+    if (statuses.isEmpty()) {
+      return Collections.emptyList();
+    }
+
+    Configuration config = ContextUtil.getConfiguration(jobContext);
+    List<Footer> footers = new ArrayList<Footer>(statuses.size());
+    Set<FileStatus> missingStatuses = new HashSet<FileStatus>();
+    Map<Path, FileStatusWrapper> missingStatusesMap =
+            new HashMap<Path, FileStatusWrapper>(missingStatuses.size());
+
+    if (footersCache == null) {
+      footersCache =
+              new LruCache<FileStatusWrapper, FootersCacheValue>(Math.max(statuses.size(),
MIN_FOOTER_CACHE_SIZE));
+    }
+    for (FileStatus status : statuses) {
+      FileStatusWrapper statusWrapper = new FileStatusWrapper(status);
+      FootersCacheValue cacheEntry =
+              footersCache.getCurrentValue(statusWrapper);
+      if (Log.DEBUG) {
+        LOG.debug("Cache entry " + (cacheEntry == null ? "not " : "")
+                + " found for '" + status.getPath() + "'");
+      }
+      if (cacheEntry != null) {
+        footers.add(cacheEntry.getFooter());
+      } else {
+        missingStatuses.add(status);
+        missingStatusesMap.put(status.getPath(), statusWrapper);
+      }
+    }
+    if (Log.DEBUG) {
+      LOG.debug("found " + footers.size() + " footers in cache and adding up "
+              + "to " + missingStatuses.size() + " missing footers to the cache");
+    }
+
+
+    if (missingStatuses.isEmpty()) {
+      return footers;
+    }
+
+    List<Footer> newFooters =
+            getFooters(config, new ArrayList<FileStatus>(missingStatuses));
+    for (Footer newFooter : newFooters) {
+      // Use the original file status objects to make sure we store a
+      // conservative (older) modification time (i.e. in case the files and
+      // footers were modified and it's not clear which version of the footers
+      // we have)
+      FileStatusWrapper fileStatus = missingStatusesMap.get(newFooter.getFile());
+      footersCache.put(fileStatus, new FootersCacheValue(fileStatus, newFooter));
     }
 
+    footers.addAll(newFooters);
     return footers;
   }
 
@@ -459,7 +512,7 @@ public class ParquetInputFormat<T> extends FileInputFormat<Void,
T> {
    * @throws IOException
    */
   public List<Footer> getFooters(Configuration configuration, List<FileStatus>
statuses) throws IOException {
-    LOG.debug("reading " + statuses.size() + " files");
+    if (Log.DEBUG) LOG.debug("reading " + statuses.size() + " files");
     return ParquetFileReader.readAllFootersInParallelUsingSummaryFiles(configuration, statuses);
   }
 
@@ -472,4 +525,80 @@ public class ParquetInputFormat<T> extends FileInputFormat<Void,
T> {
     return ParquetFileWriter.getGlobalMetaData(getFooters(jobContext));
   }
 
+  /**
+   * A simple wrapper around {@link parquet.hadoop.Footer} that also includes a
+   * modification time associated with that footer.  The modification time is
+   * used to determine whether the footer is still current.
+   */
+  static final class FootersCacheValue
+          implements LruCache.Value<FileStatusWrapper, FootersCacheValue> {
+    private final long modificationTime;
+    private final Footer footer;
+
+    public FootersCacheValue(FileStatusWrapper status, Footer footer) {
+      this.modificationTime = status.getModificationTime();
+      this.footer = new Footer(footer.getFile(), footer.getParquetMetadata());
+    }
+
+    @Override
+    public boolean isCurrent(FileStatusWrapper key) {
+      long currentModTime = key.getModificationTime();
+      boolean isCurrent = modificationTime >= currentModTime;
+      if (Log.DEBUG && !isCurrent) {
+        LOG.debug("The cache value for '" + key + "' is not current: "
+                + "cached modification time=" + modificationTime + ", "
+                + "current modification time: " + currentModTime);
+      }
+      return isCurrent;
+    }
+
+    public Footer getFooter() {
+      return footer;
+    }
+
+    @Override
+    public boolean isNewerThan(FootersCacheValue otherValue) {
+      return otherValue == null ||
+              modificationTime > otherValue.modificationTime;
+    }
+
+    public Path getPath() {
+      return footer.getFile();
+    }
+  }
+
+  /**
+   * A simple wrapper around {@link org.apache.hadoop.fs.FileStatus} with a
+   * meaningful "toString()" method
+   */
+  static final class FileStatusWrapper {
+    private final FileStatus status;
+    public FileStatusWrapper(FileStatus fileStatus) {
+      if (fileStatus == null) {
+        throw new IllegalArgumentException("FileStatus object cannot be null");
+      }
+      status = fileStatus;
+    }
+
+    public long getModificationTime() {
+      return status.getModificationTime();
+    }
+
+    @Override
+    public int hashCode() {
+      return status.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object other) {
+      return other instanceof FileStatusWrapper &&
+              status.equals(((FileStatusWrapper) other).status);
+    }
+
+    @Override
+    public String toString() {
+      return status.getPath().toString();
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/5dffe352/parquet-hadoop/src/test/java/parquet/hadoop/TestInputFormat.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/test/java/parquet/hadoop/TestInputFormat.java b/parquet-hadoop/src/test/java/parquet/hadoop/TestInputFormat.java
index 989869c..1ab1cc5 100644
--- a/parquet-hadoop/src/test/java/parquet/hadoop/TestInputFormat.java
+++ b/parquet-hadoop/src/test/java/parquet/hadoop/TestInputFormat.java
@@ -15,24 +15,27 @@
  */
 package parquet.hadoop;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.junit.Before;
 import org.junit.Test;
 import parquet.column.Encoding;
 import parquet.column.statistics.BinaryStatistics;
-import parquet.hadoop.api.ReadSupport;
 import parquet.hadoop.metadata.BlockMetaData;
 import parquet.hadoop.metadata.ColumnChunkMetaData;
 import parquet.hadoop.metadata.ColumnPath;
 import parquet.hadoop.metadata.CompressionCodecName;
 import parquet.hadoop.metadata.FileMetaData;
+import parquet.hadoop.metadata.ParquetMetadata;
 import parquet.io.ParquetDecodingException;
 import parquet.schema.MessageType;
 import parquet.schema.MessageTypeParser;
 import parquet.schema.PrimitiveType.PrimitiveTypeName;
 
+import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -40,8 +43,8 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Arrays;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.mock;
 
 public class TestInputFormat {
 
@@ -243,6 +246,49 @@ public class TestInputFormat {
     shouldSplitLengthBe(splits, 20, 20, 10, 20, 20, 10);
   }
 
+  @Test
+  public void testFooterCacheValueIsCurrent() throws IOException, InterruptedException {
+    File tempFile = getTempFile();
+    FileSystem fs = FileSystem.getLocal(new Configuration());
+    ParquetInputFormat.FootersCacheValue cacheValue = getDummyCacheValue(tempFile, fs);
+
+    assertTrue(tempFile.setLastModified(tempFile.lastModified() + 5000));
+    assertFalse(cacheValue.isCurrent(new ParquetInputFormat.FileStatusWrapper(fs.getFileStatus(new
Path(tempFile.getAbsolutePath())))));
+  }
+
+  @Test
+  public void testFooterCacheValueIsNewer() throws IOException {
+    File tempFile = getTempFile();
+    FileSystem fs = FileSystem.getLocal(new Configuration());
+    ParquetInputFormat.FootersCacheValue cacheValue = getDummyCacheValue(tempFile, fs);
+
+    assertTrue(cacheValue.isNewerThan(null));
+    assertFalse(cacheValue.isNewerThan(cacheValue));
+
+    assertTrue(tempFile.setLastModified(tempFile.lastModified() + 5000));
+    ParquetInputFormat.FootersCacheValue newerCacheValue = getDummyCacheValue(tempFile, fs);
+
+    assertTrue(newerCacheValue.isNewerThan(cacheValue));
+    assertFalse(cacheValue.isNewerThan(newerCacheValue));
+  }
+
+  private File getTempFile() throws IOException {
+    File tempFile = File.createTempFile("footer_", ".txt");
+    tempFile.deleteOnExit();
+    return tempFile;
+  }
+
+  private ParquetInputFormat.FootersCacheValue getDummyCacheValue(File file, FileSystem fs)
throws IOException {
+    Path path = new Path(file.getPath());
+    FileStatus status = fs.getFileStatus(path);
+    ParquetInputFormat.FileStatusWrapper statusWrapper = new ParquetInputFormat.FileStatusWrapper(status);
+    ParquetMetadata mockMetadata = mock(ParquetMetadata.class);
+    ParquetInputFormat.FootersCacheValue cacheValue =
+            new ParquetInputFormat.FootersCacheValue(statusWrapper, new Footer(path, mockMetadata));
+    assertTrue(cacheValue.isCurrent(statusWrapper));
+    return cacheValue;
+  }
+
   private List<ParquetInputSplit> generateSplitByMinMaxSize(long min, long max) throws
IOException {
     return ParquetInputFormat.generateSplits(
             blocks, hdfsBlocks, fileStatus, fileMetaData, schema.toString(), new HashMap<String,
String>() {{

http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/5dffe352/parquet-hadoop/src/test/java/parquet/hadoop/TestLruCache.java
----------------------------------------------------------------------
diff --git a/parquet-hadoop/src/test/java/parquet/hadoop/TestLruCache.java b/parquet-hadoop/src/test/java/parquet/hadoop/TestLruCache.java
new file mode 100644
index 0000000..91e24d8
--- /dev/null
+++ b/parquet-hadoop/src/test/java/parquet/hadoop/TestLruCache.java
@@ -0,0 +1,144 @@
+package parquet.hadoop;
+
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class TestLruCache {
+  private static final String DEFAULT_KEY = "test";
+
+  private static final class SimpleValue implements LruCache.Value<String, SimpleValue>
{
+    private boolean current;
+    private boolean newerThan;
+
+    public SimpleValue(boolean current, boolean newerThan) {
+      this.current = current;
+      this.newerThan = newerThan;
+    }
+
+    @Override
+    public boolean isCurrent(String key) {
+      return current;
+    }
+
+    public void setCurrent(boolean current) {
+      this.current = current;
+    }
+
+    @Override
+    public boolean isNewerThan(SimpleValue otherValue) {
+      return newerThan;
+    }
+
+  }
+
+  @Test
+  public void testMaxSize() {
+    LruCache<String, SimpleValue> cache = new LruCache<String, SimpleValue>(1);
+
+    String oldKey = DEFAULT_KEY;
+    String newKey = oldKey + "_new";
+
+    SimpleValue oldValue = new SimpleValue(true, true);
+    cache.put(oldKey, oldValue);
+    assertEquals(oldValue, cache.getCurrentValue(oldKey));
+    assertEquals(1, cache.size());
+
+    SimpleValue newValue = new SimpleValue(true, true);
+    cache.put(newKey, newValue);
+    assertNull(cache.getCurrentValue(oldKey));
+    assertEquals(newValue, cache.getCurrentValue(newKey));
+    assertEquals(1, cache.size());
+  }
+
+  @Test
+  public void testOlderValueIsIgnored() {
+    LruCache<String, SimpleValue> cache = new LruCache<String, SimpleValue>(1);
+
+    SimpleValue currentValue = new SimpleValue(true, true);
+    SimpleValue notAsCurrentValue = new SimpleValue(true, false);
+    cache.put(DEFAULT_KEY, currentValue);
+    cache.put(DEFAULT_KEY, notAsCurrentValue);
+    assertEquals(
+            "The existing value in the cache was overwritten",
+            currentValue,
+            cache.getCurrentValue(DEFAULT_KEY)
+    );
+  }
+
+  @Test
+  public void testOutdatedValueIsIgnored() {
+    LruCache<String, SimpleValue> cache = new LruCache<String, SimpleValue>(1);
+
+    SimpleValue outdatedValue = new SimpleValue(false, true);
+    cache.put(DEFAULT_KEY, outdatedValue);
+    assertEquals(0, cache.size());
+    assertNull(cache.getCurrentValue(DEFAULT_KEY));
+  }
+
+  @Test
+  public void testCurrentValueOverwritesExisting() {
+    LruCache<String, SimpleValue> cache = new LruCache<String, SimpleValue>(1);
+
+    SimpleValue currentValue = new SimpleValue(true, true);
+    SimpleValue notAsCurrentValue = new SimpleValue(true, false);
+    cache.put(DEFAULT_KEY, notAsCurrentValue);
+    assertEquals(1, cache.size());
+    cache.put(DEFAULT_KEY, currentValue);
+    assertEquals(1, cache.size());
+    assertEquals(
+            "The existing value in the cache was NOT overwritten",
+            currentValue,
+            cache.getCurrentValue(DEFAULT_KEY)
+    );
+  }
+
+  @Test
+  public void testGetOutdatedValueReturnsNull() {
+    LruCache<String, SimpleValue> cache = new LruCache<String, SimpleValue>(1);
+
+    SimpleValue value = new SimpleValue(true, true);
+    cache.put(DEFAULT_KEY, value);
+    assertEquals(1, cache.size());
+    assertEquals(value, cache.getCurrentValue(DEFAULT_KEY));
+
+    value.setCurrent(false);
+    assertNull("The value should not be current anymore", cache.getCurrentValue(DEFAULT_KEY));
+    assertEquals(0, cache.size());
+  }
+
+  @Test
+  public void testRemove() {
+    LruCache<String, SimpleValue> cache = new LruCache<String, SimpleValue>(1);
+
+    SimpleValue value = new SimpleValue(true, true);
+    cache.put(DEFAULT_KEY, value);
+    assertEquals(1, cache.size());
+    assertEquals(value, cache.getCurrentValue(DEFAULT_KEY));
+
+    // remove the only value
+    assertEquals(value, cache.remove(DEFAULT_KEY));
+    assertNull(cache.getCurrentValue(DEFAULT_KEY));
+    assertEquals(0, cache.size());
+  }
+
+  @Test
+  public void testClear() {
+    LruCache<String, SimpleValue> cache = new LruCache<String, SimpleValue>(2);
+
+    String key1 = DEFAULT_KEY + 1;
+    String key2 = DEFAULT_KEY + 2;
+    SimpleValue value = new SimpleValue(true, true);
+    cache.put(key1, value);
+    cache.put(key2, value);
+    assertEquals(value, cache.getCurrentValue(key1));
+    assertEquals(value, cache.getCurrentValue(key2));
+    assertEquals(2, cache.size());
+
+    cache.clear();
+    assertNull(cache.getCurrentValue(key1));
+    assertNull(cache.getCurrentValue(key2));
+    assertEquals(0, cache.size());
+  }
+
+}


Mime
View raw message