[
https://issues.apache.org/jira/browse/PHOENIX-2417?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15103518#comment-15103518
]
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_r49939293
--- Diff:
phoenix-core/src/main/java/org/apache/phoenix/schema/stats/GuidePostsInfoWriter.java
---
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.schema.stats;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.CodecUtils;
+import org.apache.phoenix.util.PrefixByteEncoder;
+import org.apache.phoenix.util.TrustedByteArrayOutputStream;
+
+/*
+ * Writer to help in writing guidePosts and creating guidePostInfo. This
is used when we are collecting stats or reading stats for a table.
+ */
+
+public class GuidePostsInfoWriter {
+ private PrefixByteEncoder encoder;
+ private byte[] lastRow;
+ private ImmutableBytesWritable guidePosts=new
ImmutableBytesWritable(ByteUtil.EMPTY_BYTE_ARRAY);
+ private long byteCount = 0;
+ private int guidePostsCount;
+
+ /**
+ * The rowCount that is flattened across the total number of guide
posts.
+ */
+ private long rowCount = 0;
+
+ /**
+ * Maximum length of a guidePost collected
+ */
+ private int maxLength;
+ private DataOutputStream output;
+ private TrustedByteArrayOutputStream stream;
+
+ public final static GuidePostsInfo EMPTY_GUIDEPOST = new
GuidePostsInfo(0,
+ new ImmutableBytesWritable(ByteUtil.EMPTY_BYTE_ARRAY), 0, 0,
0);
+
+ public int getMaxLength() {
+ return maxLength;
+ }
+ public GuidePostsInfoWriter(){
+ this.stream = new TrustedByteArrayOutputStream(1);
+ this.output = new DataOutputStream(stream);
+ this.encoder=new PrefixByteEncoder();
+ lastRow = ByteUtil.EMPTY_BYTE_ARRAY;
+ }
+
+ /**
+ * 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 writeGuidePosts( byte[] row, long byteCount, long
rowCount) {
+ if (row.length != 0 && Bytes.compareTo(lastRow, row) < 0) {
+ try {
+ 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 writeGuidePosts(byte[] row){
+ return writeGuidePosts(row, 0, 0);
+ }
+
+ public boolean writeGuidePosts(byte[] row, long byteCount){
+ return writeGuidePosts(row, byteCount, 0);
+ }
+
+ public void close() {
+ CodecUtils.close(stream);
+ }
+
+ public void incrementRowCount() {
+ this.rowCount++;
+ }
+
+ public long getByteCount() {
+ return byteCount;
+ }
+
+ public int getGuidePostsCount() {
+ return guidePostsCount;
+ }
+
+ public long getRowCount() {
+ return rowCount;
+ }
+
+ public ImmutableBytesWritable getGuidePosts() {
+ this.guidePosts.set(stream.getBuffer(), 0, stream.size());
+ return guidePosts;
+ }
+
+ public GuidePostsInfo createGuidePostInfo(){
--- End diff --
Rename to build() and call close() at end.
> 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)