[
https://issues.apache.org/jira/browse/PHOENIX-2417?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15102939#comment-15102939
]
ASF GitHub Bot commented on PHOENIX-2417:
-----------------------------------------
Github user JamesRTaylor commented on a diff in the pull request:
https://github.com/apache/phoenix/pull/147#discussion_r49925890
--- Diff:
phoenix-core/src/main/java/org/apache/phoenix/schema/stats/GuidePostsInfo.java
---
@@ -79,67 +96,66 @@ public long getRowCount() {
public void incrementRowCount() {
this.rowCount++;
}
-
- /**
- * Combines the GuidePosts per region into one.
- * @param oldInfo
- */
- public void combine(GuidePostsInfo oldInfo) {
- if (!oldInfo.getGuidePosts().isEmpty()) {
- byte[] newFirstKey = oldInfo.getGuidePosts().get(0);
- byte[] existingLastKey;
- if (!this.getGuidePosts().isEmpty()) {
- existingLastKey =
this.getGuidePosts().get(this.getGuidePosts().size() - 1);
- } else {
- existingLastKey = HConstants.EMPTY_BYTE_ARRAY;
- }
- int size = oldInfo.getGuidePosts().size();
- // If the existing guidePosts is lesser than the new
RegionInfo that we are combining
- // then add the new Region info to the end of the current
GuidePosts.
- // If the new region info is smaller than the existing
guideposts then add the existing
- // guide posts after the new guideposts.
- List<byte[]> newTotalGuidePosts = new
ArrayList<byte[]>(this.getGuidePosts().size() + size);
- if (Bytes.compareTo(existingLastKey, newFirstKey) <= 0) {
- newTotalGuidePosts.addAll(this.getGuidePosts());
- newTotalGuidePosts.addAll(oldInfo.getGuidePosts());
- } else {
- newTotalGuidePosts.addAll(oldInfo.getGuidePosts());
- newTotalGuidePosts.addAll(this.getGuidePosts());
- }
- this.guidePosts = ImmutableList.copyOf(newTotalGuidePosts);
- }
- this.byteCount += oldInfo.getByteCount();
- this.keyByteSize += oldInfo.keyByteSize;
- this.rowCount += oldInfo.getRowCount();
- }
+ public int getGuidePostsCount() {
+ return guidePostsCount;
+ }
+
/**
* The guide posts, rowCount and byteCount are accumulated every time
a guidePosts depth is
* reached while collecting stats.
* @param row
* @param byteCount
* @return
+ * @throws IOException
*/
- public boolean addGuidePost(byte[] row, long byteCount, long rowCount)
{
- if (guidePosts.isEmpty() || Bytes.compareTo(row,
guidePosts.get(guidePosts.size() - 1)) > 0) {
- List<byte[]> newGuidePosts =
Lists.newArrayListWithExpectedSize(this.getGuidePosts().size() + 1);
- newGuidePosts.addAll(guidePosts);
- newGuidePosts.add(row);
- this.guidePosts = ImmutableList.copyOf(newGuidePosts);
- this.byteCount += byteCount;
- this.keyByteSize += row.length;
- this.rowCount+=rowCount;
- return true;
+ public boolean encodeAndCollectGuidePost(byte[] row, long byteCount,
long rowCount) {
+ if (row.length != 0 && Bytes.compareTo(lastRow, row) < 0) {
+ try {
+ if(!isStreamInitialized){
+ stream = new
TrustedByteArrayOutputStream(guidePosts.getLength());
+ output = new DataOutputStream(stream);
+
stream.write(ByteUtil.copyKeyBytesIfNecessary(guidePosts));
+ encoder = new PrefixByteEncoder();
+ isStreamInitialized=true;
+ }
+ encoder.encode(output, row, 0, row.length);
+ this.byteCount += byteCount;
+ this.guidePostsCount++;
+ this.maxLength = encoder.getMaxLength();
+ this.rowCount += rowCount;
+ lastRow = row;
+ return true;
+ } catch (IOException e) {
+ return false;
+ }
}
return false;
}
-
- public boolean addGuidePost(byte[] row) {
- return addGuidePost(row, 0, 0);
+
+ public boolean encodeAndCollectGuidePost(byte[] row){
+ return encodeAndCollectGuidePost(row, 0, 0);
}
- public boolean addGuidePost(byte[] row, long byteCount) {
- return addGuidePost(row, byteCount, 0);
+ public boolean encodeAndCollectGuidePost(byte[] row, long byteCount){
+ return encodeAndCollectGuidePost(row, byteCount, 0);
}
+ public void close() {
--- End diff --
This would move to the GuidePostsInfoWriter class or maybe be unnecessary
at all as the stream would be created outside of the class.
> Compress memory used by row key byte[] of guideposts
> ----------------------------------------------------
>
> Key: PHOENIX-2417
> URL: https://issues.apache.org/jira/browse/PHOENIX-2417
> Project: Phoenix
> Issue Type: Sub-task
> Reporter: James Taylor
> Assignee: Ankit Singhal
> Fix For: 4.7.0
>
> Attachments: PHOENIX-2417.patch, PHOENIX-2417_encoder.diff,
> PHOENIX-2417_v2_wip.patch
>
>
> We've found that smaller guideposts are better in terms of minimizing any
> increase in latency for point scans. However, this increases the amount of
> memory significantly when caching the guideposts on the client. Guidepost are
> equidistant row keys in the form of raw byte[] which are likely to have a
> large percentage of their leading bytes in common (as they're stored in
> sorted order. We should use a simple compression technique to mitigate this.
> I noticed that Apache Parquet has a run length encoding - perhaps we can use
> that.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)