[ 
https://issues.apache.org/jira/browse/PHOENIX-1453?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14259200#comment-14259200
 ] 

James Taylor commented on PHOENIX-1453:
---------------------------------------

Thanks for the patch, [~ramkrishna]. I like your idea of introducing a new 
.proto file called GudePostsInfo as that improves our abstraction. Here's some 
other feedback:
- This call doesn't look right. You need to pass both the column name and the 
column type. Please update and manually test that when a 4.2.2 client that 
already has the SYSTEM.CATALOG table connects to a 4.3 server, that it's 
updated as expected.
{code}
+                                addColumnsIfNotExists(metaConnection, 
+                                        SYSTEM_CATALOG_SCHEMA + ".\"" + 
SYSTEM_STATS_TABLE, MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP, 
+                                        
PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT);
{code}
- Minor nit: remove this TODO in QueryConstants.java:
{code}
+            // TODO : Handle these new columns while table creation
{code}
- In GuidePostInfo, you need to maintain the rowCountArray and byteCountArray 
as parallel arrays to totalGudePostsList, otherwise the client will not be able 
to correlate them based on row key. The GuidePostInfo constructor should be 
this instead:
{code}
+    public GuidePostsInfo(long byteCount, List<byte[]> guidePosts, long 
rowCount) {
+        this.totalGuidePostsList = ImmutableList.copyOf(guidePosts);
+        this.byteCountArray = new long[guidePosts.size()];
+        this.rowCountArray = new long[byteCountArray.length];
+        Arrays.fill(this.byteCountArray, byteCount);
+        Arrays.fill(this.rowCountArray, rowCount);
     }
{code}
and GuidePostInfo.combine must combine both arrays together. Also, we don't 
know that the byte[] keys from the oldInfo will all be bigger (i.e. sort after) 
the existing keys for this. We do know that each lists of keys are sorted, so 
if we don't know, we may need a check here that determines the order in which 
the lists/arrays are concatenated. And a minor nit, but better to size the temp 
ArrayList correctly up front (based on the combined list size).
{code}
+    public void combine(GuidePostsRegionInfo oldInfo) {
+        // TODO: conditionally add this.getGuidePosts() first if last key is 
smaller than oldInfo first key
+        List<byte[]> oldGuidePosts = new 
ArrayList<byte[]>(oldInfo.getGuidePosts());
+        oldGuidePosts.addAll(ImmutableList.copyOf(this.getGuidePosts()));
+        this.totalGuidePostsList = ImmutableList.copyOf(oldGuidePosts);
+        // TODO: combine byteCountArray and rowCountArray similarly
{code}
- Remove System.out.println calls.
{code}
+                System.out.println("The cf is "
+                        + Bytes.toString(kv.getFamilyArray(), 
kv.getFamilyOffset(), kv.getFamilyLength()));
{code}
- Minor (but I know [~lhofhansl] will find this :-) ), don't allocate the 
rowTracker list if cachedGps != null in StatisticsCollector. Just have an if 
outside the loop instead. 
{code}
+        if (cachedGps == null) {
+            for (GuidePostsRegionInfo s : rowTracker) {
+                s.incrementRowCount();
+            }
+        } else {
+            cachedGps.getSecond().incrementRowCount();
{code}
- Please remove any commented out code, like this in StatisticsCollector:
{code}
+    //private Map<ImmutableBytesPtr, Boolean> familyMap = Maps.newHashMap();
{code}
- The code in splitStats can be improved a bit (and I know that's code that I 
wrote originally :-) ). Instead of dividing by 2 (for both row count and byte 
count), multiple by the (index + 1) / sizeOfArray. So, in other words, you'll 
be taking a percentage of each part, based on where the split is being done.
{code}
+               Cell rowCountCell = 
result.getColumnLatestCell(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, 
PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT_BYTES);
+            long rowCount = 0;
+            if (rowCountCell != null) {
+                rowCount = 
PLong.INSTANCE.getCodec().decodeLong(rowCountCell.getValueArray(),
+                        rowCountCell.getValueOffset(), SortOrder.getDefault()) 
/ 2;
+            }
{code}
- Remove comment and System.out.println
{code}
+            // Write as long_array
+            put.add(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, 
PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT_BYTES,
+                    timeStamp, PLong.INSTANCE.toBytes(gp.getRowCount()));
+            System.out.println("The row count is "+gp.getRowCount());
{code}
- Revert this comment only change:
{code}
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java 
b/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
index a3fee72..237cf36 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
@@ -221,6 +221,8 @@ public class UpgradeUtil {
                     return true;
                 }
                 // We can detect upgrade from 4.2.0 -> 4.2.1 based on the 
timestamp of the table row
+                // TODO : should we check some thing here after upgrading 
MIN_SYSTEM_TABLE_TIMESTAMP for
+                // SYSTEM.STATS table
                 if (oldTable.getTimeStamp() == 
MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP-1) {
                     byte[] oldSeqNum = 
PLong.INSTANCE.toBytes(oldTable.getSequenceNumber());
                     KeyValue seqNumKV = KeyValueUtil.newKeyValue(seqTableKey, 
{code}
- Put the doSetup() call in StatsCollectorAbstractIT instead of having it twice 
in StatsCollectorWithSplitsAndMultiCFIT and StatsCollectorIT.
- Make this assert stronger. For example, the row count array and byte count 
array must have the same length as the guidepost keys - these are parallel 
arrays, so if they don't, there's no correlation we can do between them. Also, 
figure out what the row count should be for each region, and do an assertEquals 
for the value. I'd rename TestUtil.getPTableRef(), as it's not returning a 
TableRef. How about TestUtil.getGuidePostsInfoList()?
- Remove any @Category annotations in tests. This annotation only needs to be 
set at the base test classes and will be inherited by all the derived classes.
- Add more testing in StatisticsCollectorIT. We should make sure that the row 
counts and byte counts arrays are the same size as the guide posts key list and 
we should assert the row counts are correct.


> Collect row counts per region in stats table
> --------------------------------------------
>
>                 Key: PHOENIX-1453
>                 URL: https://issues.apache.org/jira/browse/PHOENIX-1453
>             Project: Phoenix
>          Issue Type: Sub-task
>            Reporter: James Taylor
>            Assignee: ramkrishna.s.vasudevan
>         Attachments: Phoenix-1453.patch, Phoenix-1453_1.patch, 
> Phoenix-1453_10.patch, Phoenix-1453_2.patch, Phoenix-1453_3.patch, 
> Phoenix-1453_7.patch, Phoenix-1453_8.patch
>
>
> We currently collect guideposts per equal chunk, but we should also capture 
> row counts. Should we have a parallel array with the guideposts that count 
> rows per guidepost, or is it enough to have a per region count?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to