[
https://issues.apache.org/jira/browse/PHOENIX-7106?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17805969#comment-17805969
]
ASF GitHub Bot commented on PHOENIX-7106:
-----------------------------------------
kadirozde commented on code in PR #1736:
URL: https://github.com/apache/phoenix/pull/1736#discussion_r1450147986
##########
phoenix-core-client/src/main/java/org/apache/phoenix/util/ByteUtil.java:
##########
@@ -469,6 +474,105 @@ public static byte[] previousKey(byte[] key) {
return previousKey;
}
+ public static byte[] getLargestPossibleRowKeyInRange(byte[] startKey,
byte[] endKey) {
+ if (startKey.length == 0 && endKey.length == 0) {
+ return HConstants.EMPTY_END_ROW;
+ }
+ byte[] rowKey;
+ try {
+ if (startKey.length > 0 && endKey.length > 0) {
+ int commonBytesIdx = 0;
+ while (commonBytesIdx < startKey.length && commonBytesIdx <
endKey.length) {
+ if (startKey[commonBytesIdx] == endKey[commonBytesIdx]) {
+ commonBytesIdx++;
+ } else {
+ break;
+ }
+ }
+ if (commonBytesIdx == 0) {
+ rowKey =
ByteUtil.previousKeyWithLength(ByteUtil.concat(endKey,
+ new byte[startKey.length + 1]),
+ Math.max(endKey.length, startKey.length) + 1);
+ } else {
+ byte[] newStartKey;
+ byte[] newEndKey;
+ if (commonBytesIdx < startKey.length) {
+ newStartKey = new byte[startKey.length -
commonBytesIdx];
+ System.arraycopy(startKey, commonBytesIdx,
newStartKey, 0,
+ newStartKey.length);
+ } else {
+ newStartKey = startKey;
+ }
+ if (commonBytesIdx < endKey.length) {
+ newEndKey = new byte[endKey.length - commonBytesIdx];
+ System.arraycopy(endKey, commonBytesIdx, newEndKey, 0,
newEndKey.length);
+ } else {
+ newEndKey = endKey;
+ }
+ byte[] commonBytes = new byte[commonBytesIdx];
+ System.arraycopy(startKey, 0, commonBytes, 0,
commonBytesIdx);
+ byte[] tmpRowKey =
ByteUtil.previousKeyWithLength(ByteUtil.concat(newEndKey,
+ new byte[newStartKey.length + 1]),
+ Math.max(newEndKey.length, newStartKey.length) +
1);
+ // tmpRowKey can be null if newEndKey has only \x00 bytes
+ if (tmpRowKey == null) {
+ tmpRowKey = new byte[newEndKey.length - 1];
+ System.arraycopy(newEndKey, 0, tmpRowKey, 0,
tmpRowKey.length);
+ rowKey = ByteUtil.concat(commonBytes, tmpRowKey);
+ } else {
+ rowKey = ByteUtil.concat(commonBytes, tmpRowKey);
+ }
+ }
+ } else if (endKey.length > 0) {
+ rowKey = ByteUtil.previousKeyWithLength(ByteUtil.concat(endKey,
+ new byte[1]), endKey.length + 1);
+ } else {
+ rowKey = ByteUtil.nextKeyWithLength(ByteUtil.concat(startKey,
+ new byte[1]), startKey.length + 1);
+ }
+ if (rowKey == null) {
Review Comment:
Should we suppress the error by returning null or generate a
RuntimeException?
> Data Integrity issues due to invalid rowkeys returned by various coprocessors
> -----------------------------------------------------------------------------
>
> Key: PHOENIX-7106
> URL: https://issues.apache.org/jira/browse/PHOENIX-7106
> Project: Phoenix
> Issue Type: Improvement
> Affects Versions: 5.2.0, 5.1.4
> Reporter: Viraj Jasani
> Assignee: Viraj Jasani
> Priority: Blocker
> Fix For: 5.2.0, 5.1.4
>
>
> HBase scanner interface expects server to perform scan of the cells from
> HFile or Block cache and return consistent data i.e. rowkey of the cells
> returned should stay in the range of the scan boundaries. When a region moves
> and scanner needs reset, or if the current row is too large and the server
> returns partial row, the subsequent scanner#next is supposed to return
> remaining cells. When this happens, cell rowkeys returned by servers i.e. any
> coprocessors is expected to be in the scan boundary range so that server can
> reliably perform its validation and return remaining cells as expected.
> Phoenix client initiates serial or parallel scans from the aggregators based
> on the region boundaries and the scan boundaries are sometimes adjusted based
> on where optimizer provided key ranges, to include tenant boundaries, salt
> boundaries etc. After the client opens the scanner and performs scan
> operation, some of the coprocs return invalid rowkey for the following cases:
> # Grouped aggregate queries
> # Some Ungrouped aggregate queries
> # Offset queries
> # Dummy cells returned with empty rowkey
> # Update statistics queries
> # Uncovered Index queries
> # Ordered results at server side
> # ORDER BY DESC on rowkey
> # Global Index read-repair
> # Paging region scanner with HBase scanner reopen
> # ORDER BY on non-pk column(s) with/without paging
> # GROUP BY on non-pk column(s) with/without paging
> Since many of these cases return reserved rowkeys, they are likely not going
> to match scan or region boundaries. It has potential to cause data integrity
> issues in certain scenarios as explained above. Empty rowkey returned by
> server can be treated as end of the region scan by HBase client.
> With the paging feature enabled, if the page size is kept low, we have higher
> chances of scanners returning dummy cell, resulting in increased num of RPC
> calls for better latency and timeouts. We should return only valid rowkey in
> the scan range for all the cases where we perform above mentioned operations
> like complex aggregate or offset queries etc.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)