[
https://issues.apache.org/jira/browse/PHOENIX-2417?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15102173#comment-15102173
]
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_r49882278
--- Diff:
phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java ---
@@ -1003,26 +1003,16 @@ public static PTable
createFromProto(PTableProtos.PTable table) {
boolean isImmutableRows = table.getIsImmutableRows();
SortedMap<byte[], GuidePostsInfo> tableGuidePosts = new
TreeMap<byte[], GuidePostsInfo>(Bytes.BYTES_COMPARATOR);
for (PTableProtos.PTableStats pTableStatsProto :
table.getGuidePostsList()) {
- List<byte[]> value =
Lists.newArrayListWithExpectedSize(pTableStatsProto.getValuesCount());
- for (int j = 0; j < pTableStatsProto.getValuesCount(); j++) {
- value.add(pTableStatsProto.getValues(j).toByteArray());
- }
- // No op
- pTableStatsProto.getGuidePostsByteCount();
- value =
Lists.newArrayListWithExpectedSize(pTableStatsProto.getValuesCount());
PGuidePosts pGuidePosts = pTableStatsProto.getPGuidePosts();
- for(int j = 0; j < pGuidePosts.getGuidePostsCount(); j++) {
- value.add(pGuidePosts.getGuidePosts(j).toByteArray());
- }
long guidePostsByteCount = pGuidePosts.getByteCount();
long rowCount = pGuidePosts.getRowCount();
- // TODO : Not exposing MIN/MAX key outside to client
- GuidePostsInfo info =
- new GuidePostsInfo(guidePostsByteCount, value,
rowCount);
+ int maxLength = pGuidePosts.getMaxLength();
+ int guidePostsCount = pGuidePosts.getGuidePostsCount();
+ GuidePostsInfo info = new GuidePostsInfo(guidePostsByteCount,
+ new
ImmutableBytesWritable(pGuidePosts.getGuidePosts().toByteArray()), rowCount,
maxLength, guidePostsCount);
--- End diff --
Use new
ImmutableBytesWritable(HBaseZeroCopyByteString.zeroCopyGetBytes(pGuidePosts.getGuidePosts()))
here (and elsewhere) instead to prevent copying the underlying byte array.
> 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)