Skip to content

Commit

Permalink
Minor javadoc and whitespace changes. Also added the FileStatusWrappe…
Browse files Browse the repository at this point in the history
…r class to ParquetInputFormat to make sure that the debugging log statements print out meaningful paths.
  • Loading branch information
matt-martin committed Jun 24, 2014
1 parent 250a398 commit 99bb5a3
Show file tree
Hide file tree
Showing 3 changed files with 153 additions and 74 deletions.
102 changes: 64 additions & 38 deletions parquet-hadoop/src/main/java/parquet/hadoop/LruCache.java
Original file line number Diff line number Diff line change
Expand Up @@ -6,13 +6,15 @@
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}.
* 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.
* @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);
Expand All @@ -25,7 +27,7 @@ final class LruCache<K, V extends LruCache.Value<K, V>> {
* 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(int maxSize) {
public LruCache(final int maxSize) {
this(maxSize, DEFAULT_LOAD_FACTOR, true);
}

Expand All @@ -34,16 +36,21 @@ public LruCache(int maxSize) {
*
* @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
* @param accessOrder the ordering mode - {@code true} for access-order,
* {@code false} for insertion-order
*/
public LruCache(final int maxSize, float loadFactor, boolean accessOrder) {
public LruCache(final int maxSize, final float loadFactor, final boolean accessOrder) {
int initialCapacity = Math.round(maxSize / loadFactor);
cacheMap =
new LinkedHashMap<K, V>(Math.round(maxSize / loadFactor), loadFactor, accessOrder) {
new LinkedHashMap<K, V>(initialCapacity, loadFactor, accessOrder) {
@Override
public boolean removeEldestEntry(Map.Entry<K,V> eldest) {
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());
if (Log.DEBUG) {
LOG.debug("Removing eldest entry in cache: "
+ eldest.getKey());
}
}
return result;
}
Expand All @@ -53,33 +60,42 @@ public boolean removeEldestEntry(Map.Entry<K,V> eldest) {
/**
* 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.
* @return the previous value associated with key, or null if there was no
* mapping for key.
*/
public V remove(K 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 + "'");
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.
* 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(K key, V newValue) {
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"));
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");
if (Log.WARN) {
LOG.warn("Ignoring new cache entry for '" + key + "' because "
+ "existing cache entry is newer");
}
return;
}

Expand All @@ -95,22 +111,26 @@ public void put(K key, V newValue) {
}

/**
* Removes all of the mappings from this cache. The cache will be empty after this call returns.
* 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.
* 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
* @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(K 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 (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);
Expand All @@ -129,27 +149,33 @@ public int 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.
* {@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 <V>
* @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)
* @return {@code true} the value is still current, {@code false} the value is no longer useful
* 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
*/
public boolean isCurrent(K key);
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
* @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.
*/
public boolean isNewerThan(V otherValue);
boolean isNewerThan(V otherValue);
}

}
117 changes: 84 additions & 33 deletions parquet-hadoop/src/main/java/parquet/hadoop/ParquetInputFormat.java
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@
*/
package parquet.hadoop;

import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
Expand Down Expand Up @@ -85,7 +84,7 @@ public class ParquetInputFormat<T> extends FileInputFormat<Void, T> {

private static final int MIN_FOOTER_CACHE_SIZE = 100;

private LruCache<FileStatus, FootersCacheValue> footersCache;
private LruCache<FileStatusWrapper, FootersCacheValue> footersCache;

private Class<?> readSupportClass;

Expand Down Expand Up @@ -453,40 +452,46 @@ public List<Footer> getFooters(JobContext jobContext) throws IOException {
Configuration config = ContextUtil.getConfiguration(jobContext);
List<Footer> footers = new ArrayList<Footer>(statuses.size());
Set<FileStatus> missingStatuses = new HashSet<FileStatus>();

if (footersCache != null) {
for (FileStatus status : statuses) {
FootersCacheValue cacheEntry = footersCache.getCurrentValue(status);
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);
}
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() + "'");
}
} else {
// initialize the cache to store all of the current statuses or a default minimum size. The sizing of this LRU
// cache was chosen to mimic prior behavior (i.e. so that performance would be at least as good as it was before)
footersCache = new LruCache<FileStatus, FootersCacheValue>(Math.max(statuses.size(), MIN_FOOTER_CACHE_SIZE));
missingStatuses.addAll(statuses);
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 (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));
Map<Path, FileStatus> missingStatusesMap = new HashMap<Path, FileStatus>(missingStatuses.size());
for (FileStatus missingStatus : missingStatuses) {
missingStatusesMap.put(missingStatus.getPath(), missingStatus);
}
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)
FileStatus fileStatus = missingStatusesMap.get(newFooter.getFile());
// 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));
}

Expand Down Expand Up @@ -515,34 +520,80 @@ public GlobalMetaData getGlobalMetaData(JobContext jobContext) throws IOExceptio
return ParquetFileWriter.getGlobalMetaData(getFooters(jobContext));
}

static final class FootersCacheValue implements LruCache.Value<FileStatus, FootersCacheValue> {
/**
* 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(FileStatus status, Footer footer) {
public FootersCacheValue(FileStatusWrapper status, Footer footer) {
this.modificationTime = status.getModificationTime();
this.footer = new Footer(footer.getFile(), footer.getParquetMetadata());
}

public boolean isCurrent(FileStatus key) {
@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.getPath() + "' is not current: " +
"cached modification time=" + modificationTime + ", current modification time: " + 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;
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();
}
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -253,7 +253,7 @@ public void testFooterCacheValueIsCurrent() throws IOException, InterruptedExcep
ParquetInputFormat.FootersCacheValue cacheValue = getDummyCacheValue(tempFile, fs);

assertTrue(tempFile.setLastModified(tempFile.lastModified() + 5000));
assertFalse(cacheValue.isCurrent(fs.getFileStatus(new Path(tempFile.getAbsolutePath()))));
assertFalse(cacheValue.isCurrent(new ParquetInputFormat.FileStatusWrapper(fs.getFileStatus(new Path(tempFile.getAbsolutePath())))));
}

@Test
Expand Down Expand Up @@ -281,9 +281,11 @@ private File getTempFile() throws IOException {
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(status, new Footer(path, mockMetadata));
assertTrue(cacheValue.isCurrent(status));
ParquetInputFormat.FootersCacheValue cacheValue =
new ParquetInputFormat.FootersCacheValue(statusWrapper, new Footer(path, mockMetadata));
assertTrue(cacheValue.isCurrent(statusWrapper));
return cacheValue;
}

Expand Down

0 comments on commit 99bb5a3

Please sign in to comment.