[
https://issues.apache.org/jira/browse/PHOENIX-1333?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14165347#comment-14165347
]
James Taylor commented on PHOENIX-1333:
---------------------------------------
Thanks for the patch, [~ramkrishna]. Some feedback:
- Instead of storing the GUIDE_POSTS_WIDTH as the config param, let's store the
total number of bytes traversed, because it's possible that the depth will be
different for different regions. So when you traverse to combine all the
guideposts, keep a running count (i.e. guidePostDepth * numGuidePosts). You'll
still need to capture the number of guideposts too, so instead of
Pair<Pair<Integer, Long>, byte[]>, how about creating a simple Java object for
this: GuidePostInfo with a int guidePostCount, long guidePostDepth, and byte[]
guidePostBytes.
- Make sure when you combine the guideposts for a given family, that you add
the previous guidePostCount and guidePostDepth to the new one here:
{code}
+ Pair<Integer, byte[]> pair = new Pair<Integer,byte[]>();
+ List<byte[]> guidePosts = gps.getSecond();
+ if (!guidePosts.isEmpty()) {
+ WritableUtils.writeVInt(os, guidePosts.size());
+ for (byte[] element : guidePosts) {
+ WritableUtils.writeVInt(os, element.length);
+ os.write(element);
+ }
+ pair.setFirst(guidePosts.size());
+ // Should this be closed before converting to ByteArray
+ pair.setSecond(bs.toByteArray());
+ return pair;
}
{code}
- Same thing in StatisticsUtil.readStatistics. Make sure you keep the running
total correctly. Then you'll want to add the total of guidePostDepth and
guidePostCount to the PStats object and add accessors in PTable for them -
we'll want this available on the client side.
- For the sizing of the TrustedByteArrayOutputStream, just create it inside the
if statement, once you've found a match. I'd keep a running count of the number
of bytes and serialize it in the front of the byte[] and use it for sizing
(that plus the writeVInt size of the count and you should be able to get an
exact byte amount. Then at the end, instead of always doing bs.toByteArray(),
use ByteUtil.copyKeyBytesIfNecessary() as you should be able to prevent
another copy of this pretty sizable byte array.
{code}
+ public Pair<Integer,byte[]> getGuidePosts(String fam) {
if (!guidePostsMap.isEmpty()) {
- Pair<Integer,List<byte[]>> gps = guidePostsMap.get(fam);
- if (gps != null) {
- List<byte[]> guidePosts = gps.getSecond();
- if (!guidePosts.isEmpty()) {
- byte[][] array = new byte[guidePosts.size()][];
- int i = 0;
- for (byte[] element : guidePosts) {
- array[i] = element;
- i++;
+ // TODO : What can be the initial size
+ TrustedByteArrayOutputStream bs = new
TrustedByteArrayOutputStream(1024);
{code}
> Store statistics guideposts as VARBINARY
> ----------------------------------------
>
> Key: PHOENIX-1333
> URL: https://issues.apache.org/jira/browse/PHOENIX-1333
> Project: Phoenix
> Issue Type: Sub-task
> Reporter: James Taylor
> Assignee: ramkrishna.s.vasudevan
> Priority: Critical
> Attachments: Phoenix-1333.patch
>
>
> There's a potential problem with storing the guideposts as a VARBINARY ARRAY,
> as pointed out by PHOENIX-1329. We'd run into this issue if we're collecting
> stats for a table with a trailing VARBINARY row key column if the value
> contained embedded null bytes. Because of this, we're better off storing
> guideposts as VARBINARY and serializing/deserializing in the following manner:
> <byte length as vint><bytes><byte length as vint><bytes>...
> We should also store as a separate KeyValue column the total number of
> guideposts. So the schema of SYSTEM.STATS would look like this now instead:
> {code}
> public static final String CREATE_STATS_TABLE_METADATA =
> "CREATE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" +
> SYSTEM_STATS_TABLE + "\"(\n" +
> // PK columns
> PHYSICAL_NAME + " VARCHAR NOT NULL," +
> COLUMN_FAMILY + " VARCHAR," +
> REGION_NAME + " VARCHAR," +
> GUIDE_POSTS + " VARBINARY," +
> GUIDE_POSTS_COUNT + " SMALLINT," +
> MIN_KEY + " VARBINARY," +
> MAX_KEY + " VARBINARY," +
> LAST_STATS_UPDATE_TIME+ " DATE, "+
> "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY ("
> + PHYSICAL_NAME + ","
> + COLUMN_FAMILY + ","+ REGION_NAME+"))\n" +
> // TODO: should we support versioned stats?
> // Install split policy to prevent a physical table's stats from
> being split across regions.
> HTableDescriptor.SPLIT_POLICY + "='" +
> MetaDataSplitPolicy.class.getName() + "'\n";
> {code}
> Then the serialization code in StatisticsTable.addStats() would need to
> change to populate the GUIDE_POSTS_COUNT and serialize the GUIDE_POSTS in the
> new format.
> The deserialization code is isolated to StatisticsUtil.readStatisitics(). It
> would need to read the GUIDE_POSTS_COUNT first for estimated sizing, and then
> deserialize the GUIDE_POSTS in the new format.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)