You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@parquet.apache.org by fo...@apache.org on 2020/01/10 12:55:25 UTC
[parquet-mr] branch master updated: PARQUET-1729: Avoid AutoBoxing
in EncodingStats (#717)
This is an automated email from the ASF dual-hosted git repository.
fokko pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/parquet-mr.git
The following commit(s) were added to refs/heads/master by this push:
new d85a8f5 PARQUET-1729: Avoid AutoBoxing in EncodingStats (#717)
d85a8f5 is described below
commit d85a8f5dcfc1381655fcccaa81a2e83ba812f6a4
Author: belugabehr <12...@users.noreply.github.com>
AuthorDate: Fri Jan 10 07:55:15 2020 -0500
PARQUET-1729: Avoid AutoBoxing in EncodingStats (#717)
* PARQUET-1729: Avoid AutoBoxing in EncodingStats
* Updated unit tests to more properly check stats value
---
.../org/apache/parquet/column/EncodingStats.java | 29 ++++++++---------
.../apache/parquet/column/TestEncodingStats.java | 36 +++++++++++-----------
2 files changed, 33 insertions(+), 32 deletions(-)
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/EncodingStats.java b/parquet-column/src/main/java/org/apache/parquet/column/EncodingStats.java
index d775145..5d05838 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/EncodingStats.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/EncodingStats.java
@@ -25,6 +25,7 @@ import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.Map;
import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
import static org.apache.parquet.column.Encoding.PLAIN_DICTIONARY;
import static org.apache.parquet.column.Encoding.RLE_DICTIONARY;
@@ -36,12 +37,12 @@ import static org.apache.parquet.column.Encoding.RLE_DICTIONARY;
* convenience methods for those checks, like {@link #hasNonDictionaryEncodedPages()}.
*/
public class EncodingStats {
- final Map<Encoding, Integer> dictStats;
- final Map<Encoding, Integer> dataStats;
+ final Map<Encoding, Number> dictStats;
+ final Map<Encoding, Number> dataStats;
private final boolean usesV2Pages;
- private EncodingStats(Map<Encoding, Integer> dictStats,
- Map<Encoding, Integer> dataStats,
+ private EncodingStats(Map<Encoding, Number> dictStats,
+ Map<Encoding, Number> dataStats,
boolean usesV2Pages) {
this.dictStats = dictStats;
this.dataStats = dataStats;
@@ -57,13 +58,13 @@ public class EncodingStats {
}
public int getNumDictionaryPagesEncodedAs(Encoding enc) {
- final Integer i = dictStats.get(enc);
- return (i == null) ? 0 : i.intValue();
+ final Number pageCount = dictStats.get(enc);
+ return (pageCount == null) ? 0 : pageCount.intValue();
}
public int getNumDataPagesEncodedAs(Encoding enc) {
- final Integer i = dataStats.get(enc);
- return (i == null) ? 0 : i.intValue();
+ final Number pageCount = dataStats.get(enc);
+ return (pageCount == null) ? 0 : pageCount.intValue();
}
public boolean hasDictionaryPages() {
@@ -103,8 +104,8 @@ public class EncodingStats {
* Used to build {@link EncodingStats} from metadata or to accumulate stats as pages are written.
*/
public static class Builder {
- private final Map<Encoding, Integer> dictStats = new LinkedHashMap<>();
- private final Map<Encoding, Integer> dataStats = new LinkedHashMap<>();
+ private final Map<Encoding, AtomicInteger> dictStats = new LinkedHashMap<>();
+ private final Map<Encoding, AtomicInteger> dataStats = new LinkedHashMap<>();
private boolean usesV2Pages = false;
public Builder clear() {
@@ -124,8 +125,8 @@ public class EncodingStats {
}
public Builder addDictEncoding(Encoding encoding, int numPages) {
- Integer pages = dictStats.get(encoding);
- dictStats.put(encoding, numPages + (pages != null ? pages : 0));
+ dictStats.computeIfAbsent(encoding, enc -> new AtomicInteger(0))
+ .addAndGet(numPages);
return this;
}
@@ -141,8 +142,8 @@ public class EncodingStats {
}
public Builder addDataEncoding(Encoding encoding, int numPages) {
- Integer pages = dataStats.get(encoding);
- dataStats.put(encoding, numPages + (pages != null ? pages : 0));
+ dataStats.computeIfAbsent(encoding, enc -> new AtomicInteger(0))
+ .addAndGet(numPages);
return this;
}
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/TestEncodingStats.java b/parquet-column/src/test/java/org/apache/parquet/column/TestEncodingStats.java
index 434da33..3dd4ee6 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/TestEncodingStats.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/TestEncodingStats.java
@@ -21,9 +21,6 @@ package org.apache.parquet.column;
import org.junit.Test;
-import java.util.HashMap;
-import java.util.Map;
-
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
@@ -39,12 +36,6 @@ public class TestEncodingStats {
builder.addDataEncoding(Encoding.DELTA_BYTE_ARRAY);
EncodingStats stats1 = builder.build();
- Map<Encoding, Integer> expectedDictStats1 = new HashMap<>();
- expectedDictStats1.put(Encoding.PLAIN, 1);
- Map<Encoding, Integer> expectedDataStats1 = new HashMap<>();
- expectedDataStats1.put(Encoding.RLE_DICTIONARY, 3);
- expectedDataStats1.put(Encoding.DELTA_BYTE_ARRAY, 2);
-
builder.clear();
builder.addDataEncoding(Encoding.PLAIN);
builder.addDataEncoding(Encoding.PLAIN);
@@ -52,15 +43,24 @@ public class TestEncodingStats {
builder.addDataEncoding(Encoding.PLAIN);
EncodingStats stats2 = builder.build();
- Map<Encoding, Integer> expectedDictStats2 = new HashMap<>();
- Map<Encoding, Integer> expectedDataStats2 = new HashMap<>();
- expectedDataStats2.put(Encoding.PLAIN, 4);
-
- assertEquals("Dictionary stats should be correct", expectedDictStats2, stats2.dictStats);
- assertEquals("Data stats should be correct", expectedDataStats2, stats2.dataStats);
-
- assertEquals("Dictionary stats should be correct after reuse", expectedDictStats1, stats1.dictStats);
- assertEquals("Data stats should be correct after reuse", expectedDataStats1, stats1.dataStats);
+ assertEquals("Dictionary stats should be correct", 0,
+ stats2.dictStats.size());
+ assertEquals("Data stats size should be correct", 1,
+ stats2.dataStats.size());
+ assertEquals("Data stats content should be correct", 4,
+ stats2.dataStats.get(Encoding.PLAIN).intValue());
+
+ assertEquals("Dictionary stats size should be correct after reuse",
+ 1, stats1.dictStats.size());
+ assertEquals("Dictionary stats content should be correct", 1,
+ stats1.dictStats.get(Encoding.PLAIN).intValue());
+
+ assertEquals("Data stats size should be correct after reuse", 2,
+ stats1.dataStats.size());
+ assertEquals("Data stats content should be correct", 3,
+ stats1.dataStats.get(Encoding.RLE_DICTIONARY).intValue());
+ assertEquals("Data stats content should be correct", 2,
+ stats1.dataStats.get(Encoding.DELTA_BYTE_ARRAY).intValue());
}
@Test